Introduces Feeds 2.0
commit c3f577861fc705d848c1641605689cadd6973bae
Merge: ebc4cae fc0c2c0
Author: ramangrover29 <ramangrover29@gmail.com>
Date: Fri Jun 26 13:04:05 2015 -0700
Merge branch 'raman/feeds_2_release' of https://code.google.com/p/asterixdb-sandbox into raman/feeds_2_release
Conflicts:
asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedServlet.java
asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectAccessors.java
commit ebc4cae21a7302869f953df1ebda601e798d12d2
Author: ramangrover29 <ramangrover29@gmail.com>
Date: Sat Jun 20 17:14:45 2015 -0700
Introduces Feeds 2.0
Some of the prominent chnages introduced are as follows
a) Support for building a cascade network of feeds (via secondary feeds feature)
b) Feed Management Console for tracking active feeds and associated metrics
c) Support for elastic runtime for data ingestion
d) Improved fault-tolerance with support for logging of failed records
Documentation has been added at asterix-doc/src/site/markdown/feeds/
commit fc0c2c0549a6ee8b202e57607d2e110478cd57bb
Author: ramangrover29 <ramangrover29@gmail.com>
Date: Sat Jun 20 17:14:45 2015 -0700
Introduces Feeds 2.0
Some of the prominent chnages introduced are as follows
a) Support for building a cascade network of feeds (via secondary feeds feature)
b) Feed Management Console for tracking active feeds and associated metrics
c) Support for elastic runtime for data ingestion
d) Improved fault-tolerance with support for logging of failed records
Documentation has been added at asterix-doc/src/site/markdown/feeds/
Change-Id: I498f01c591a229aaf51cec43ab20f3e5c4f072f4
Reviewed-on: https://asterix-gerrit.ics.uci.edu/297
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Steven Jacobs <sjaco002@ucr.edu>
Reviewed-by: abdullah alamoudi <bamousaa@gmail.com>
diff --git a/asterix-algebra/pom.xml b/asterix-algebra/pom.xml
index 7c138b3..f3db662 100644
--- a/asterix-algebra/pom.xml
+++ b/asterix-algebra/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>
@@ -21,14 +17,14 @@
</parent>
<artifactId>asterix-algebra</artifactId>
- <licenses>
- <license>
- <name>Apache License, Version 2.0</name>
- <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
- <distribution>repo</distribution>
- <comments>A business-friendly OSS license</comments>
- </license>
- </licenses>
+ <licenses>
+ <license>
+ <name>Apache License, Version 2.0</name>
+ <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+ <distribution>repo</distribution>
+ <comments>A business-friendly OSS license</comments>
+ </license>
+ </licenses>
<build>
<plugins>
@@ -61,7 +57,8 @@
</plugins>
<pluginManagement>
<plugins>
- <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+ <!--This plugin's configuration is used to store Eclipse m2e settings
+ only. It has no influence on the Maven build itself. -->
<plugin>
<groupId>org.eclipse.m2e</groupId>
<artifactId>lifecycle-mapping</artifactId>
@@ -121,12 +118,12 @@
<version>0.8.7-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
- <dependency>
- <groupId>edu.uci.ics.asterix</groupId>
- <artifactId>asterix-transactions</artifactId>
- <version>0.8.7-SNAPSHOT</version>
- <scope>compile</scope>
- </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.asterix</groupId>
+ <artifactId>asterix-transactions</artifactId>
+ <version>0.8.7-SNAPSHOT</version>
+ <scope>compile</scope>
+ </dependency>
<dependency>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>algebricks-compiler</artifactId>
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
index 4411869..dcbc70c 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataLookupPOperator.java
@@ -109,7 +109,7 @@
@Override
public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
throws AlgebricksException {
- AqlDataSource ds = new DatasetDataSource(datasetId, datasetId.getDataverseName(), datasetId.getDatasetName(),
+ AqlDataSource ds = new DatasetDataSource(datasetId, datasetId.getDataverseName(), datasetId.getDatasourceName(),
recordType, AqlDataSourceType.EXTERNAL_DATASET);
IDataSourcePropertiesProvider dspp = ds.getPropertiesProvider();
AbstractScanOperator as = (AbstractScanOperator) op;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/AnalysisUtil.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/AnalysisUtil.java
index 83f137e..e89ff51 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/AnalysisUtil.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/AnalysisUtil.java
@@ -125,7 +125,7 @@
public static Pair<String, String> getDatasetInfo(AbstractDataSourceOperator op) throws AlgebricksException {
AqlSourceId srcId = (AqlSourceId) op.getDataSource().getId();
- return new Pair<String, String>(srcId.getDataverseName(), srcId.getDatasetName());
+ return new Pair<String, String>(srcId.getDataverseName(), srcId.getDatasourceName());
}
private static List<FunctionIdentifier> fieldAccessFunctions = new ArrayList<FunctionIdentifier>();
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 b90442f..10ab856 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
@@ -18,6 +18,7 @@
import java.util.LinkedList;
import java.util.List;
+import edu.uci.ics.asterix.optimizer.rules.AddEquivalenceClassForRecordConstructorRule;
import edu.uci.ics.asterix.optimizer.rules.AsterixInlineVariablesRule;
import edu.uci.ics.asterix.optimizer.rules.AsterixIntroduceGroupByCombinerRule;
import edu.uci.ics.asterix.optimizer.rules.ByNameToByIndexFieldAccessRule;
@@ -32,7 +33,6 @@
import edu.uci.ics.asterix.optimizer.rules.FeedScanCollectionToUnnest;
import edu.uci.ics.asterix.optimizer.rules.FuzzyEqRule;
import edu.uci.ics.asterix.optimizer.rules.IfElseToSwitchCaseFunctionRule;
-import edu.uci.ics.asterix.optimizer.rules.AddEquivalenceClassForRecordConstructorRule;
import edu.uci.ics.asterix.optimizer.rules.InlineUnnestFunctionRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceAutogenerateIDRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceDynamicTypeCastForExternalFunctionRule;
@@ -40,6 +40,7 @@
import edu.uci.ics.asterix.optimizer.rules.IntroduceEnforcedListTypeRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceInstantLockSearchCallbackRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceMaterializationForInsertWithSelfScanRule;
+import edu.uci.ics.asterix.optimizer.rules.IntroduceRandomPartitioningFeedComputationRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceRapidFrameFlushProjectAssignRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceSecondaryIndexInsertDeleteRule;
import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastForInsertRule;
@@ -310,6 +311,7 @@
physicalRewritesTopLevel.add(new SetAlgebricksPhysicalOperatorsRule());
physicalRewritesTopLevel.add(new IntroduceRapidFrameFlushProjectAssignRule());
physicalRewritesTopLevel.add(new SetExecutionModeRule());
+ physicalRewritesTopLevel.add(new IntroduceRandomPartitioningFeedComputationRule());
return physicalRewritesTopLevel;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
new file mode 100644
index 0000000..c0ccd30
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
@@ -0,0 +1,98 @@
+/*
+ * 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 org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
+import edu.uci.ics.asterix.metadata.declared.FeedDataSource;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+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.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.RandomPartitionPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class IntroduceRandomPartitioningFeedComputationRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ ILogicalOperator op = opRef.getValue();
+ if (!op.getOperatorTag().equals(LogicalOperatorTag.ASSIGN)) {
+ return false;
+ }
+
+ ILogicalOperator opChild = op.getInputs().get(0).getValue();
+ if (!opChild.getOperatorTag().equals(LogicalOperatorTag.DATASOURCESCAN)) {
+ return false;
+ }
+
+ DataSourceScanOperator scanOp = (DataSourceScanOperator) opChild;
+ AqlDataSource dataSource = (AqlDataSource) scanOp.getDataSource();
+ if (!dataSource.getDatasourceType().equals(AqlDataSourceType.FEED)) {
+ return false;
+ }
+
+ final FeedDataSource feedDataSource = (FeedDataSource) dataSource;
+ Feed feed = feedDataSource.getFeed();
+ if (feed.getAppliedFunction() == null) {
+ return false;
+ }
+
+ ExchangeOperator exchangeOp = new ExchangeOperator();
+ INodeDomain domain = new INodeDomain() {
+ @Override
+ public boolean sameAs(INodeDomain domain) {
+ return domain == this;
+ }
+
+ @Override
+ public Integer cardinality() {
+ return feedDataSource.getComputeCardinality();
+ }
+ };
+
+ exchangeOp.setPhysicalOperator(new RandomPartitionPOperator(domain));
+ op.getInputs().get(0).setValue(exchangeOp);
+ exchangeOp.getInputs().add(new MutableObject<ILogicalOperator>(scanOp));
+ ExecutionMode em = ((AbstractLogicalOperator) scanOp).getExecutionMode();
+ exchangeOp.setExecutionMode(em);
+ exchangeOp.computeDeliveredPhysicalProperties(context);
+ context.computeAndSetTypeEnvironmentForOperator(exchangeOp);
+
+ AssignOperator assignOp = (AssignOperator) opRef.getValue();
+ AssignPOperator assignPhyOp = (AssignPOperator) assignOp.getPhysicalOperator();
+ assignPhyOp.setCardinalityConstraint(domain.cardinality());
+
+ return true;
+ }
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ return false;
+ }
+
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 8552c39..84f7230 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -136,7 +136,7 @@
AqlDataSource datasetSource = (AqlDataSource) insertOp.getDataSource();
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
String dataverseName = datasetSource.getId().getDataverseName();
- String datasetName = datasetSource.getId().getDatasetName();
+ String datasetName = datasetSource.getId().getDatasourceName();
Dataset dataset = mp.findDataset(dataverseName, datasetName);
if (dataset == null) {
throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
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 a0d268a..1fe0c6f 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
@@ -121,9 +121,9 @@
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
AqlSourceId asid = ((IDataSource<AqlSourceId>) scan.getDataSource()).getId();
- Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasetName());
+ Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasourceName());
if (dataset == null) {
- throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
+ throw new AlgebricksException("Dataset " + asid.getDatasourceName() + " not found.");
}
if (dataset.getDatasetType() != DatasetType.INTERNAL) {
return false;
@@ -306,9 +306,9 @@
}
AqlSourceId asid = dataSource.getId();
AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
- Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasetName());
+ Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasourceName());
if (dataset == null) {
- throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
+ throw new AlgebricksException("Dataset " + asid.getDatasourceName() + " not found.");
}
if (dataset.getDatasetType() != DatasetType.INTERNAL) {
setAsFinal(access, context, finalAnnot);
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 1104c53..78a327f 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
@@ -20,7 +20,8 @@
import org.apache.commons.lang3.mutable.Mutable;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedActivity.FeedActivityDetails;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleListener.ConnectionLocation;
import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
@@ -134,27 +135,23 @@
return true;
}
- if (fid.equals(AsterixBuiltinFunctions.FEED_INGEST)) {
+ if (fid.equals(AsterixBuiltinFunctions.FEED_COLLECT)) {
if (unnest.getPositionalVariable() != null) {
throw new AlgebricksException("No positional variables are allowed over datasets.");
}
- String feedArg = getStringArgument(f, 0);
- String outputType = getStringArgument(f, 1);
- String targetDataset = getStringArgument(f, 2);
+ String dataverse = getStringArgument(f, 0);
+ String sourceFeedName = getStringArgument(f, 1);
+ String getTargetFeed = getStringArgument(f, 2);
+ String subscriptionLocation = getStringArgument(f, 3);
+ String targetDataset = getStringArgument(f, 4);
+ String outputType = getStringArgument(f, 5);
AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
- 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);
- }
- AqlSourceId asid = new AqlSourceId(dataverseName, feedName);
- String policyName = metadataProvider.getConfig().get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
- FeedPolicy policy = metadataProvider.findFeedPolicy(dataverseName, policyName);
+ AqlSourceId asid = new AqlSourceId(dataverse, getTargetFeed);
+ String policyName = metadataProvider.getConfig().get(FeedActivityDetails.FEED_POLICY_NAME);
+ FeedPolicy policy = metadataProvider.findFeedPolicy(dataverse, policyName);
if (policy == null) {
policy = BuiltinFeedPolicies.getFeedPolicy(policyName);
if (policy == null) {
@@ -165,9 +162,9 @@
ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
v.add(unnest.getVariable());
- DataSourceScanOperator scan = new DataSourceScanOperator(v, createFeedDataSource(asid,
- new FeedConnectionId(dataverseName, feedName, targetDataset), metadataProvider, policy,
- outputType));
+ String csLocations = metadataProvider.getConfig().get(FeedActivityDetails.COLLECT_LOCATIONS);
+ DataSourceScanOperator scan = new DataSourceScanOperator(v, createFeedDataSource(asid, targetDataset,
+ sourceFeedName, subscriptionLocation, metadataProvider, policy, outputType, csLocations));
List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
scanInpList.addAll(unnest.getInputs());
@@ -192,16 +189,20 @@
context.addPrimaryKey(pk);
}
- private AqlDataSource createFeedDataSource(AqlSourceId aqlId, FeedConnectionId feedId,
- AqlMetadataProvider metadataProvider, FeedPolicy feedPolicy, String outputType) throws AlgebricksException {
+ private AqlDataSource createFeedDataSource(AqlSourceId aqlId, String targetDataset, String sourceFeedName,
+ String subscriptionLocation, AqlMetadataProvider metadataProvider, FeedPolicy feedPolicy,
+ String outputType, String locations) throws AlgebricksException {
if (!aqlId.getDataverseName().equals(
metadataProvider.getDefaultDataverse() == null ? null : metadataProvider.getDefaultDataverse()
.getDataverseName())) {
return null;
}
- IAType feedOutputType = metadataProvider.findType(feedId.getDataverse(), outputType);
- FeedDataSource feedDataSource = new FeedDataSource(aqlId, feedId, feedOutputType,
- AqlDataSource.AqlDataSourceType.FEED);
+ IAType feedOutputType = metadataProvider.findType(aqlId.getDataverseName(), outputType);
+ Feed sourceFeed = metadataProvider.findFeed(aqlId.getDataverseName(), sourceFeedName);
+
+ FeedDataSource feedDataSource = new FeedDataSource(aqlId, targetDataset, feedOutputType,
+ AqlDataSource.AqlDataSourceType.FEED, sourceFeed.getFeedId(), sourceFeed.getFeedType(),
+ ConnectionLocation.valueOf(subscriptionLocation), locations.split(","));
feedDataSource.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy);
return feedDataSource;
}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
index f06784a..e00737e 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
@@ -26,6 +26,7 @@
import edu.uci.ics.asterix.aql.expression.DropStatement;
import edu.uci.ics.asterix.aql.expression.InsertStatement;
import edu.uci.ics.asterix.aql.expression.NodeGroupDropStatement;
+import edu.uci.ics.asterix.common.api.IClusterManagementWork.ClusterState;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
import edu.uci.ics.asterix.metadata.dataset.hints.DatasetHints;
@@ -48,12 +49,12 @@
public void validateOperation(Dataverse defaultDataverse, Statement stmt) throws AsterixException {
- if (!(AsterixClusterProperties.INSTANCE.getState().equals(AsterixClusterProperties.State.ACTIVE) && AsterixClusterProperties.INSTANCE
+ if (!(AsterixClusterProperties.INSTANCE.getState().equals(ClusterState.ACTIVE) && AsterixClusterProperties.INSTANCE
.isGlobalRecoveryCompleted())) {
int maxWaitCycles = AsterixAppContextInfo.getInstance().getExternalProperties().getMaxWaitClusterActive();
int waitCycleCount = 0;
try {
- while (!AsterixClusterProperties.INSTANCE.getState().equals(AsterixClusterProperties.State.ACTIVE)
+ while (!AsterixClusterProperties.INSTANCE.getState().equals(ClusterState.ACTIVE)
&& waitCycleCount < maxWaitCycles) {
Thread.sleep(1000);
waitCycleCount++;
@@ -61,21 +62,21 @@
} catch (InterruptedException e) {
if (LOGGER.isLoggable(Level.WARNING)) {
LOGGER.warning("Thread interrupted while waiting for cluster to be "
- + AsterixClusterProperties.State.ACTIVE);
+ + ClusterState.ACTIVE);
}
}
- if (!AsterixClusterProperties.INSTANCE.getState().equals(AsterixClusterProperties.State.ACTIVE)) {
- throw new AsterixException(" Asterix Cluster is in " + AsterixClusterProperties.State.UNUSABLE
+ if (!AsterixClusterProperties.INSTANCE.getState().equals(ClusterState.ACTIVE)) {
+ throw new AsterixException(" Asterix Cluster is in " + ClusterState.UNUSABLE
+ " state." + "\n One or more Node Controllers have left or haven't joined yet.\n");
} else {
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Cluster is now " + AsterixClusterProperties.State.ACTIVE);
+ LOGGER.info("Cluster is now " + ClusterState.ACTIVE);
}
}
}
- if (AsterixClusterProperties.INSTANCE.getState().equals(AsterixClusterProperties.State.UNUSABLE)) {
- throw new AsterixException(" Asterix Cluster is in " + AsterixClusterProperties.State.UNUSABLE + " state."
+ if (AsterixClusterProperties.INSTANCE.getState().equals(ClusterState.UNUSABLE)) {
+ throw new AsterixException(" Asterix Cluster is in " + ClusterState.UNUSABLE + " state."
+ "\n One or more Node Controllers have left.\n");
}
@@ -94,7 +95,7 @@
}
if (!AsterixClusterProperties.INSTANCE.isGlobalRecoveryCompleted()) {
throw new AsterixException(" Asterix Cluster Global recovery is not yet complete and The system is in "
- + AsterixClusterProperties.State.ACTIVE + " state");
+ + ClusterState.ACTIVE + " state");
}
}
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 ef9797d..0360ffb 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
@@ -31,9 +31,11 @@
import edu.uci.ics.asterix.aql.expression.CompactStatement;
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.CreateFeedPolicyStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
+import edu.uci.ics.asterix.aql.expression.CreatePrimaryFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateSecondaryFeedStatement;
import edu.uci.ics.asterix.aql.expression.DatasetDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDropStatement;
@@ -43,6 +45,7 @@
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.FeedPolicyDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -401,6 +404,14 @@
leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
break;
}
+ case SUBSCRIBE_FEED: {
+ ILogicalOperator insertOp = new InsertDeleteOperator(targetDatasource, varRef, varRefsForLoading,
+ InsertDeleteOperator.Kind.INSERT, false);
+ insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+ leafOperator = new SinkOperator();
+ leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
+ break;
+ }
}
topOp = leafOperator;
}
@@ -1547,15 +1558,22 @@
// TODO Auto-generated method stub
return null;
}
-
+
@Override
- public Pair<ILogicalOperator, LogicalVariable> visitCreateFeedStatement(CreateFeedStatement del,
+ public Pair<ILogicalOperator, LogicalVariable> visitCreatePrimaryFeedStatement(CreatePrimaryFeedStatement del,
Mutable<ILogicalOperator> arg) throws AsterixException {
// TODO Auto-generated method stub
return null;
}
@Override
+ public Pair<ILogicalOperator, LogicalVariable> visitCreateSecondaryFeedStatement(CreateSecondaryFeedStatement 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
@@ -1575,4 +1593,16 @@
// TODO Auto-generated method stub
return null;
}
+
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitCreateFeedPolicyStatement(CreateFeedPolicyStatement cfps,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ return null;
+ }
+
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDropFeedPolicyStatement(FeedPolicyDropStatement dfs,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ 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 e21eb9b..aa071f6 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
@@ -30,9 +30,12 @@
import edu.uci.ics.asterix.aql.expression.CompactStatement;
import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedPolicyStatement;
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.CreatePrimaryFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateSecondaryFeedStatement;
import edu.uci.ics.asterix.aql.expression.DatasetDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDropStatement;
@@ -42,6 +45,7 @@
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.FeedPolicyDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -1444,13 +1448,6 @@
}
@Override
- 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
@@ -1470,4 +1467,32 @@
// TODO Auto-generated method stub
return null;
}
+
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitCreatePrimaryFeedStatement(CreatePrimaryFeedStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitCreateSecondaryFeedStatement(CreateSecondaryFeedStatement del,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitCreateFeedPolicyStatement(CreateFeedPolicyStatement cfps,
+ Mutable<ILogicalOperator> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<ILogicalOperator, LogicalVariable> visitDropFeedPolicyStatement(FeedPolicyDropStatement dfs,
+ 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 6e9f197..c08f100 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
@@ -34,6 +34,7 @@
import edu.uci.ics.asterix.aql.expression.WhereClause;
import edu.uci.ics.asterix.aql.literal.StringLiteral;
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionRequest;
import edu.uci.ics.asterix.common.functions.FunctionConstants;
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
@@ -385,6 +386,48 @@
return policyName;
}
}
+
+ public static class CompiledSubscribeFeedStatement implements ICompiledDmlStatement {
+
+ private final FeedConnectionRequest request;
+ private Query query;
+ private final int varCounter;
+
+ public CompiledSubscribeFeedStatement(FeedConnectionRequest request, Query query, int varCounter) {
+ this.request = request;
+ this.query = query;
+ this.varCounter = varCounter;
+ }
+
+ @Override
+ public String getDataverseName() {
+ return request.getReceivingFeedId().getDataverse();
+ }
+
+ @Override
+ public String getDatasetName() {
+ return request.getTargetDataset();
+ }
+
+ public int getVarCounter() {
+ return varCounter;
+ }
+
+ public Query getQuery() {
+ return query;
+ }
+
+ public void setQuery(Query query) {
+ this.query = query;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.SUBSCRIBE_FEED;
+ }
+
+ }
+
public static class CompiledDisconnectFeedStatement implements ICompiledDmlStatement {
private String dataverseName;
diff --git a/asterix-app/data/twitter/obamatweets_duplicate.adm b/asterix-app/data/twitter/obamatweets_duplicate.adm
new file mode 100644
index 0000000..e8442e1
--- /dev/null
+++ b/asterix-app/data/twitter/obamatweets_duplicate.adm
@@ -0,0 +1,12 @@
+ { "id": "nc1:1", "username": "BronsonMike", "location": "", "text": "@GottaLaff @reutersus Christie and obama just foul weather friends", "timestamp": "Thu Dec 06 16:53:06 PST 2012" }
+ { "id": "nc1:100", "username": "KidrauhlProuds", "location": "", "text": "RT @01Direclieber: A filha do Michael Jackson uma Belieber,a filha do Eminem e uma Belieber,as filhas de Obama sao Beliebers, e a filha do meu pai e Belieber", "timestamp": "Thu Dec 06 16:53:16 PST 2012" }
+ { "id": "nc1:102", "username": "jaysauce82", "location": "", "text": "Not voting for President Obama #BadDecision", "timestamp": "Thu Dec 06 16:53:16 PST 2012" }
+ { "id": "nc1:104", "username": "princeofsupras", "location": "", "text": "RT @01Direclieber: A filha do Michael Jackson e uma Belieber,a filha do Eminem e uma Belieber,as filhas de Obama sao Beliebers, e a filha do meu pai e Belieber", "timestamp": "Thu Dec 06 16:53:15 PST 2012" }
+ { "id": "nc1:106", "username": "GulfDogs", "location": "", "text": "Obama Admin Knew Libyan Terrorists Had US-Provided Weaponsteaparty #tcot #ccot #NewGuards #BreitbartArmy #patriotwttp://t.co/vJxzrQUE", "timestamp": "Thu Dec 06 16:53:14 PST 2012" }
+ { "id": "nc1:108", "username": "Laugzpz", "location": "", "text": "@AlfredoJalife Maestro Obama se hace de la vista gorda, es un acuerdo de siempre creo yo.", "timestamp": "Thu Dec 06 16:53:14 PST 2012" }
+ { "id": "nc1:108", "username": "Laugzpz", "location": "", "text": "@AlfredoJalife Maestro Obama se hace de la vista gorda, es un acuerdo de siempre creo yo.", "timestamp": "Thu Dec 06 16:53:14 PST 2012" }
+ { "id": "nc1:11", "username": "magarika", "location": "", "text": "RT @ken24xavier: Obama tells SOROS - our plan is ALMOST finished http://t.co/WvzK0GtU", "timestamp": "Thu Dec 06 16:53:05 PST 2012" }
+ { "id": "nc1:111", "username": "ToucanMall", "location": "", "text": "RT @WorldWar3Watch: Michelle Obama Gets More Grammy Nominations Than Justin ... #Obama #WW3 http://t.co/0Wv2GKij", "timestamp": "Thu Dec 06 16:53:13 PST 2012" }
+ { "id": "nc1:113", "username": "ToucanMall", "location": "", "text": "RT @ObamaPalooza: Tiffany Shared What $2,000 Meant to Her ... and the President Stopped by to Talk About It http://t.co/sgT7lsNV #Obama", "timestamp": "Thu Dec 06 16:53:12 PST 2012" }
+ { "id": "nc1:115", "username": "thewildpitch", "location": "", "text": "RT @RevkahJC: Dennis Miller: Obama Should Just Say He Wants To Tax Successful People http://t.co/Ihlemy9Y", "timestamp": "Thu Dec 06 16:53:11 PST 2012" }
+ { "id": "nc1:117", "username": "Rnugent24", "location": "", "text": "RT @ConservativeQuo: unemployment is above 8% again. I wonder how long it will take for Obama to start blaming Bush? 3-2-1 #tcot #antiobama", "timestamp": "Thu Dec 06 16:53:10 PST 2012" }
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 6409a13..6d7f2a4 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
@@ -22,6 +22,7 @@
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
+import edu.uci.ics.asterix.common.config.AsterixFeedProperties;
import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
@@ -31,11 +32,11 @@
import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.feeds.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.feeds.FeedManager;
import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
-import edu.uci.ics.asterix.metadata.feeds.FeedManager;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepositoryFactory;
import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
@@ -89,6 +90,8 @@
private AsterixMetadataProperties metadataProperties;
private AsterixStorageProperties storageProperties;
private AsterixTransactionProperties txnProperties;
+ private AsterixFeedProperties feedProperties;
+
private AsterixThreadExecutor threadExecutor;
private DatasetLifecycleManager indexLifecycleManager;
@@ -111,6 +114,7 @@
metadataProperties = new AsterixMetadataProperties(ASTERIX_PROPERTIES_ACCESSOR);
storageProperties = new AsterixStorageProperties(ASTERIX_PROPERTIES_ACCESSOR);
txnProperties = new AsterixTransactionProperties(ASTERIX_PROPERTIES_ACCESSOR);
+ feedProperties = new AsterixFeedProperties(ASTERIX_PROPERTIES_ACCESSOR);
}
public void initialize() throws IOException, ACIDException, AsterixException {
@@ -147,7 +151,8 @@
isShuttingdown = false;
- feedManager = new FeedManager(ncApplicationContext.getNodeId());
+ feedManager = new FeedManager(ncApplicationContext.getNodeId(), feedProperties,
+ compilerProperties.getFrameSize());
// The order of registration is important. The buffer cache must registered before recovery and transaction managers.
ILifeCycleComponentManager lccm = ncApplicationContext.getLifeCycleComponentManager();
@@ -234,6 +239,11 @@
public AsterixExternalProperties getExternalProperties() {
return externalProperties;
}
+
+ @Override
+ public AsterixFeedProperties getFeedProperties() {
+ return feedProperties;
+ }
@Override
public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/FeedWorkCollection.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/FeedWorkCollection.java
new file mode 100644
index 0000000..e206d27
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/FeedWorkCollection.java
@@ -0,0 +1,197 @@
+/*
+ * 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.common;
+
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.api.common.SessionConfig.OutputFormat;
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.expression.DataverseDecl;
+import edu.uci.ics.asterix.aql.expression.Identifier;
+import edu.uci.ics.asterix.aql.expression.SubscribeFeedStatement;
+import edu.uci.ics.asterix.aql.translator.AqlTranslator;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionRequest;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionRequest.ConnectionStatus;
+import edu.uci.ics.asterix.common.feeds.api.IFeedWork;
+import edu.uci.ics.asterix.common.feeds.api.IFeedWorkEventListener;
+import edu.uci.ics.asterix.feeds.FeedCollectInfo;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+/**
+ * A collection of feed management related task, each represented as an implementation of {@code IFeedWork}.
+ */
+public class FeedWorkCollection {
+
+ private static Logger LOGGER = Logger.getLogger(FeedWorkCollection.class.getName());
+
+ /**
+ * The task of subscribing to a feed to obtain data.
+ */
+ public static class SubscribeFeedWork implements IFeedWork {
+
+ private final Runnable runnable;
+
+ private final FeedConnectionRequest request;
+
+ @Override
+ public Runnable getRunnable() {
+ return runnable;
+ }
+
+ public SubscribeFeedWork(String[] locations, FeedConnectionRequest request) {
+ this.runnable = new SubscribeFeedWorkRunnable(locations, request);
+ this.request = request;
+ }
+
+ private static class SubscribeFeedWorkRunnable implements Runnable {
+
+ private final FeedConnectionRequest request;
+ private final String[] locations;
+
+ public SubscribeFeedWorkRunnable(String[] locations, FeedConnectionRequest request) {
+ this.request = request;
+ this.locations = locations;
+ }
+
+ @Override
+ public void run() {
+ try {
+ PrintWriter writer = new PrintWriter(System.out, true);
+ SessionConfig pc = new SessionConfig(writer, OutputFormat.ADM);
+ DataverseDecl dataverseDecl = new DataverseDecl(new Identifier(request.getReceivingFeedId()
+ .getDataverse()));
+ SubscribeFeedStatement subscribeStmt = new SubscribeFeedStatement(locations, request);
+ List<Statement> statements = new ArrayList<Statement>();
+ statements.add(dataverseDecl);
+ statements.add(subscribeStmt);
+ AqlTranslator translator = new AqlTranslator(statements, pc);
+ translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null, AqlTranslator.ResultDelivery.SYNC);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Submitted connection requests for execution: " + request);
+ }
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Exception in executing " + request);
+ }
+ throw new RuntimeException(e);
+ }
+ }
+ }
+
+ public static class FeedSubscribeWorkEventListener implements IFeedWorkEventListener {
+
+ @Override
+ public void workFailed(IFeedWork work, Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning(" Feed subscription request " + ((SubscribeFeedWork) work).request
+ + " failed with exception " + e);
+ }
+ }
+
+ @Override
+ public void workCompleted(IFeedWork work) {
+ ((SubscribeFeedWork) work).request.setSubscriptionStatus(ConnectionStatus.ACTIVE);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.warning(" Feed subscription request " + ((SubscribeFeedWork) work).request + " completed ");
+ }
+ }
+
+ }
+
+ public FeedConnectionRequest getRequest() {
+ return request;
+ }
+
+ @Override
+ public String toString() {
+ return "SubscribeFeedWork for [" + request + "]";
+ }
+
+ }
+
+ /**
+ * The task of activating a set of feeds.
+ */
+ public static class ActivateFeedWork implements IFeedWork {
+
+ private final Runnable runnable;
+
+ @Override
+ public Runnable getRunnable() {
+ return runnable;
+ }
+
+ public ActivateFeedWork(List<FeedCollectInfo> feedsToRevive) {
+ this.runnable = new FeedsActivateRunnable(feedsToRevive);
+ }
+
+ public ActivateFeedWork() {
+ this.runnable = new FeedsActivateRunnable();
+ }
+
+ private static class FeedsActivateRunnable implements Runnable {
+
+ private List<FeedCollectInfo> feedsToRevive;
+ private Mode mode;
+
+ public enum Mode {
+ REVIVAL_POST_NODE_REJOIN
+ }
+
+ public FeedsActivateRunnable(List<FeedCollectInfo> feedsToRevive) {
+ this.feedsToRevive = feedsToRevive;
+ }
+
+ public FeedsActivateRunnable() {
+ }
+
+ @Override
+ public void run() {
+ switch (mode) {
+ 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 (FeedCollectInfo 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());
+ }
+ }
+ }
+ }
+ }
+
+ }
+
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/ConnectorAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/ConnectorAPIServlet.java
index c5fb76b..7559326 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/ConnectorAPIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/ConnectorAPIServlet.java
@@ -28,6 +28,7 @@
import org.json.JSONException;
import org.json.JSONObject;
+import edu.uci.ics.asterix.feeds.CentralFeedManager;
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
@@ -76,7 +77,7 @@
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
// Retrieves file splits of the dataset.
- AqlMetadataProvider metadataProvider = new AqlMetadataProvider(null);
+ AqlMetadataProvider metadataProvider = new AqlMetadataProvider(null, CentralFeedManager.getInstance());
metadataProvider.setMetadataTxnContext(mdTxnCtx);
Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
if (dataset == null) {
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 51af387..555ee67 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
@@ -33,7 +33,7 @@
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_FEED };
+ Kind.CREATE_PRIMARY_FEED, Kind.CREATE_SECONDARY_FEED, Kind.DROP_FEED, Kind.CREATE_FEED_POLICY, Kind.DROP_FEED_POLICY };
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
deleted file mode 100644
index 5cb53b2..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDashboardServlet.java
+++ /dev/null
@@ -1,133 +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.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.common.feeds.FeedConnectionId;
-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;
-
-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
deleted file mode 100644
index 463ce01..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/FeedDataProviderServlet.java
+++ /dev/null
@@ -1,118 +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.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.common.feeds.FeedConnectionId;
-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;
-
-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
index 96b3c31..adca4c6 100644
--- 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
@@ -21,19 +21,20 @@
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.io.PrintWriter;
-import java.util.List;
+import java.util.Collection;
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.common.feeds.FeedActivity;
+import edu.uci.ics.asterix.common.feeds.FeedActivity.FeedActivityDetails;
import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
-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.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLoadManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.feeds.CentralFeedManager;
public class FeedServlet extends HttpServlet {
private static final long serialVersionUID = 1L;
@@ -50,73 +51,116 @@
resourcePath = requestURI;
}
- try {
- InputStream is = FeedServlet.class.getResourceAsStream(resourcePath);
- if (is == null) {
- response.sendError(HttpServletResponse.SC_NOT_FOUND);
- return;
- }
+ 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")) {
+ // 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) {
+ 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 {
+ Collection<FeedActivity> lfa = CentralFeedManager.getInstance().getFeedLoadManager().getFeedActivities();
+ StringBuilder ldStr = new StringBuilder();
+ ldStr.append("<br />");
+ ldStr.append("<br />");
+ if (lfa == null || lfa.isEmpty()) {
+ ldStr.append("Currently there are no active feeds in AsterixDB");
+ } else {
+ ldStr.append("Active Feeds");
+ }
+ insertTable(ldStr, lfa);
+ outStr = String.format(sb.toString(), ldStr.toString());
+
+ }
+
+ PrintWriter out = response.getWriter();
+ out.println(outStr);
+ }
+
+ private void insertTable(StringBuilder html, Collection<FeedActivity> list) {
+ html.append("<table style=\"width:100%\">");
+ html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_FEED_NAME + "</th>");
+ html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_DATASET_NAME + "</th>");
+ html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_ACTIVE_SINCE + "</th>");
+ html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_INTAKE_STAGE + "</th>");
+ html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_COMPUTE_STAGE + "</th>");
+ html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_STORE_STAGE + "</th>");
+ html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_INTAKE_RATE + "</th>");
+ html.append("<th>" + FeedServletUtil.Constants.TABLE_HEADER_STORE_RATE + "</th>");
+ for (FeedActivity activity : list) {
+ insertRow(html, activity);
+ }
+ }
+
+ private void insertRow(StringBuilder html, FeedActivity activity) {
+ String intake = activity.getFeedActivityDetails().get(FeedActivityDetails.INTAKE_LOCATIONS);
+ String compute = activity.getFeedActivityDetails().get(FeedActivityDetails.COMPUTE_LOCATIONS);
+ String store = activity.getFeedActivityDetails().get(FeedActivityDetails.STORAGE_LOCATIONS);
+
+ IFeedLoadManager loadManager = CentralFeedManager.getInstance().getFeedLoadManager();
+ FeedConnectionId connectionId = new FeedConnectionId(new FeedId(activity.getDataverseName(),
+ activity.getFeedName()), activity.getDatasetName());
+ int intakeRate = loadManager.getOutflowRate(connectionId, FeedRuntimeType.COLLECT) * intake.split(",").length;
+ int storeRate = loadManager.getOutflowRate(connectionId, FeedRuntimeType.STORE) * store.split(",").length;
+
+ html.append("<tr>");
+ html.append("<td>" + activity.getFeedName() + "</td>");
+ html.append("<td>" + activity.getDatasetName() + "</td>");
+ html.append("<td>" + activity.getConnectTimestamp() + "</td>");
+ //html.append("<td>" + insertLink(html, FeedDashboardServlet.getParameterizedURL(activity), "Details") + "</td>");
+ html.append("<td>" + intake + "</td>");
+ html.append("<td>" + compute + "</td>");
+ html.append("<td>" + store + "</td>");
+ String color = "black";
+ if (intakeRate > storeRate) {
+ color = "red";
+ }
+ if (intakeRate < 0) {
+ html.append("<td>" + "UNKNOWN" + "</td>");
+ } else {
+ html.append("<td>" + insertColoredText("" + intakeRate, color) + " rec/sec" + "</td>");
+ }
+ if (storeRate < 0) {
+ html.append("<td>" + "UNKNOWN" + "</td>");
+ } else {
+ html.append("<td>" + insertColoredText("" + storeRate, color) + " rec/sec" + "</td>");
+ }
+ html.append("</tr>");
+ }
+
+ private String insertLink(StringBuilder html, String url, String displayText) {
+ return ("<a href=\"" + url + "\">" + displayText + "</a>");
+ }
+
+ private String insertColoredText(String s, String color) {
+ return "<font color=\"" + color + "\">" + s + "</font>";
}
}
+
+
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
index ff29a23..8567810 100644
--- 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
@@ -9,7 +9,7 @@
import java.util.logging.Logger;
import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
-import edu.uci.ics.asterix.hyracks.bootstrap.FeedLifecycleListener;
+import edu.uci.ics.asterix.feeds.FeedLifecycleListener;
import edu.uci.ics.asterix.metadata.feeds.RemoteSocketMessageListener;
public class FeedServletUtil {
@@ -17,6 +17,17 @@
private static final Logger LOGGER = Logger.getLogger(FeedServletUtil.class.getName());
private static final char EOL = (char) "\n".getBytes()[0];
+ public static final class Constants {
+ public static final String TABLE_HEADER_FEED_NAME = "Feed";
+ public static final String TABLE_HEADER_DATASET_NAME = "Dataset";
+ public static final String TABLE_HEADER_ACTIVE_SINCE = "Timestamp";
+ public static final String TABLE_HEADER_INTAKE_STAGE = "Intake Stage";
+ public static final String TABLE_HEADER_COMPUTE_STAGE = "Compute Stage";
+ public static final String TABLE_HEADER_STORE_STAGE = "Store Stage";
+ public static final String TABLE_HEADER_INTAKE_RATE = "Intake";
+ public static final String TABLE_HEADER_STORE_RATE = "Store";
+ }
+
public static void initiateSubscription(FeedConnectionId feedId, String host, int port) throws IOException {
LinkedBlockingQueue<String> outbox = new LinkedBlockingQueue<String>();
int subscriptionPort = port + 1;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryResultAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryResultAPIServlet.java
index 3f104fe..bf7ec75 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryResultAPIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/QueryResultAPIServlet.java
@@ -25,9 +25,7 @@
import org.json.JSONArray;
import org.json.JSONObject;
-import edu.uci.ics.asterix.api.common.APIFramework;
import edu.uci.ics.asterix.api.common.SessionConfig;
-import edu.uci.ics.asterix.api.common.SessionConfig.OutputFormat;
import edu.uci.ics.asterix.result.ResultReader;
import edu.uci.ics.asterix.result.ResultUtils;
import edu.uci.ics.hyracks.api.client.HyracksConnection;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java
index 4e8427d..dd19c97 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java
@@ -15,7 +15,6 @@
package edu.uci.ics.asterix.api.http.servlet;
import java.io.IOException;
-import java.io.PrintWriter;
import java.io.StringWriter;
import java.nio.charset.StandardCharsets;
import java.util.List;
@@ -30,7 +29,6 @@
import org.apache.commons.io.IOUtils;
import org.json.JSONObject;
-import edu.uci.ics.asterix.api.common.APIFramework;
import edu.uci.ics.asterix.api.common.SessionConfig;
import edu.uci.ics.asterix.api.common.SessionConfig.OutputFormat;
import edu.uci.ics.asterix.aql.base.Statement;
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 e5ddf2b..dad5975 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
@@ -16,6 +16,8 @@
import java.io.BufferedReader;
import java.io.File;
+import java.io.FileInputStream;
+import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.PrintWriter;
import java.rmi.RemoteException;
@@ -26,6 +28,8 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
import java.util.Random;
import java.util.Set;
import java.util.logging.Level;
@@ -44,9 +48,12 @@
import edu.uci.ics.asterix.aql.expression.CompactStatement;
import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedPolicyStatement;
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.CreatePrimaryFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateSecondaryFeedStatement;
import edu.uci.ics.asterix.aql.expression.DatasetDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDecl;
import edu.uci.ics.asterix.aql.expression.DataverseDropStatement;
@@ -55,6 +62,7 @@
import edu.uci.ics.asterix.aql.expression.DropStatement;
import edu.uci.ics.asterix.aql.expression.ExternalDetailsDecl;
import edu.uci.ics.asterix.aql.expression.FeedDropStatement;
+import edu.uci.ics.asterix.aql.expression.FeedPolicyDropStatement;
import edu.uci.ics.asterix.aql.expression.FunctionDecl;
import edu.uci.ics.asterix.aql.expression.FunctionDropStatement;
import edu.uci.ics.asterix.aql.expression.IDatasetDetailsDecl;
@@ -69,6 +77,7 @@
import edu.uci.ics.asterix.aql.expression.RefreshExternalDatasetStatement;
import edu.uci.ics.asterix.aql.expression.RunStatement;
import edu.uci.ics.asterix.aql.expression.SetStatement;
+import edu.uci.ics.asterix.aql.expression.SubscribeFeedStatement;
import edu.uci.ics.asterix.aql.expression.TypeDecl;
import edu.uci.ics.asterix.aql.expression.TypeDropStatement;
import edu.uci.ics.asterix.aql.expression.TypeExpression;
@@ -82,8 +91,21 @@
import edu.uci.ics.asterix.common.config.GlobalConfig;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedActivity.FeedActivityDetails;
import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionRequest;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedJointKey;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.feeds.api.IFeedJoint;
+import edu.uci.ics.asterix.common.feeds.api.IFeedJoint.FeedJointType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleEventSubscriber;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleEventSubscriber.FeedLifecycleEvent;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleListener.ConnectionLocation;
import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.feeds.CentralFeedManager;
+import edu.uci.ics.asterix.feeds.FeedJoint;
+import edu.uci.ics.asterix.feeds.FeedLifecycleListener;
import edu.uci.ics.asterix.file.DatasetOperations;
import edu.uci.ics.asterix.file.DataverseOperations;
import edu.uci.ics.asterix.file.ExternalIndexingOperations;
@@ -106,14 +128,17 @@
import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.metadata.entities.Feed;
-import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.Feed.FeedType;
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.entities.PrimaryFeed;
+import edu.uci.ics.asterix.metadata.entities.SecondaryFeed;
+import edu.uci.ics.asterix.metadata.feeds.FeedLifecycleEventSubscriber;
import edu.uci.ics.asterix.metadata.feeds.FeedUtil;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapterFactory;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.metadata.utils.ExternalDatasetsRegistry;
import edu.uci.ics.asterix.metadata.utils.MetadataLockManager;
@@ -138,6 +163,7 @@
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledInsertStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
+import edu.uci.ics.asterix.translator.CompiledStatements.CompiledSubscribeFeedStatement;
import edu.uci.ics.asterix.translator.CompiledStatements.ICompiledDmlStatement;
import edu.uci.ics.asterix.translator.TypeTranslator;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -229,7 +255,8 @@
for (Statement stmt : aqlStatements) {
validateOperation(activeDefaultDataverse, stmt);
- AqlMetadataProvider metadataProvider = new AqlMetadataProvider(activeDefaultDataverse);
+ AqlMetadataProvider metadataProvider = new AqlMetadataProvider(activeDefaultDataverse,
+ CentralFeedManager.getInstance());
metadataProvider.setWriterFactory(writerFactory);
metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
metadataProvider.setOutputFile(outputFile);
@@ -307,7 +334,8 @@
break;
}
- case CREATE_FEED: {
+ case CREATE_PRIMARY_FEED:
+ case CREATE_SECONDARY_FEED: {
handleCreateFeedStatement(metadataProvider, stmt, hcc);
break;
}
@@ -316,6 +344,12 @@
handleDropFeedStatement(metadataProvider, stmt, hcc);
break;
}
+
+ case DROP_FEED_POLICY: {
+ handleDropFeedPolicyStatement(metadataProvider, stmt, hcc);
+ break;
+ }
+
case CONNECT_FEED: {
handleConnectFeedStatement(metadataProvider, stmt, hcc);
break;
@@ -326,6 +360,16 @@
break;
}
+ case SUBSCRIBE_FEED: {
+ handleSubscribeFeedStatement(metadataProvider, stmt, hcc);
+ break;
+ }
+
+ case CREATE_FEED_POLICY: {
+ handleCreateFeedPolicyStatement(metadataProvider, stmt, hcc);
+ break;
+ }
+
case QUERY: {
metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
metadataProvider.setResultAsyncMode(resultDelivery == ResultDelivery.ASYNC
@@ -468,7 +512,7 @@
ProgressState progress = ProgressState.NO_PROGRESS;
DatasetDecl dd = (DatasetDecl) stmt;
- String dataverseName = getActiveDataverseName(dd.getDataverse());
+ String dataverseName = getActiveDataverse(dd.getDataverse());
String datasetName = dd.getName().getValue();
DatasetType dsType = dd.getDatasetType();
String itemTypeName = dd.getItemTypeName().getValue();
@@ -647,6 +691,27 @@
}
}
+ private void validateIfResourceIsActiveInFeed(String dataverseName, String datasetName) throws AsterixException {
+ List<FeedConnectionId> activeFeedConnections = FeedLifecycleListener.INSTANCE.getActiveFeedConnections(null);
+ boolean resourceInUse = false;
+ StringBuilder builder = new StringBuilder();
+
+ if (activeFeedConnections != null && !activeFeedConnections.isEmpty()) {
+ for (FeedConnectionId connId : activeFeedConnections) {
+ if (connId.getDatasetName().equals(datasetName)) {
+ resourceInUse = true;
+ builder.append(connId + "\n");
+ }
+ }
+ }
+
+ if (resourceInUse) {
+ throw new AsterixException("Dataset " + datasetName + " is currently being "
+ + "fed into by the following feed(s).\n" + builder.toString() + "\n" + "Operation not supported");
+ }
+
+ }
+
private String getNodeGroupName(Identifier ngNameId, DatasetDecl dd, String dataverse) {
if (ngNameId != null) {
return ngNameId.getValue();
@@ -716,7 +781,7 @@
IHyracksClientConnection hcc) throws Exception {
ProgressState progress = ProgressState.NO_PROGRESS;
CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
- String dataverseName = getActiveDataverseName(stmtCreateIndex.getDataverseName());
+ String dataverseName = getActiveDataverse(stmtCreateIndex.getDataverseName());
String datasetName = stmtCreateIndex.getDatasetName().getValue();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -826,19 +891,7 @@
}
if (ds.getDatasetType() == DatasetType.INTERNAL) {
- 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.");
- }
-
+ validateIfResourceIsActiveInFeed(dataverseName, datasetName);
} else {
// External dataset
// Check if the dataset is indexible
@@ -1070,7 +1123,7 @@
private void handleCreateTypeStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
TypeDecl stmtCreateType = (TypeDecl) stmt;
- String dataverseName = getActiveDataverseName(stmtCreateType.getDataverseName());
+ String dataverseName = getActiveDataverse(stmtCreateType.getDataverseName());
String typeName = stmtCreateType.getIdent().getValue();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -1130,21 +1183,26 @@
}
//# disconnect all feeds from any datasets in the dataverse.
- List<FeedActivity> feedActivities = MetadataManager.INSTANCE.getActiveFeeds(mdTxnCtx, dataverseName, null);
+ List<FeedConnectionId> activeFeedConnections = FeedLifecycleListener.INSTANCE
+ .getActiveFeedConnections(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()));
- try {
- handleDisconnectFeedStatement(metadataProvider, disStmt, hcc);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Disconnected feed " + fa.getFeedName() + " from dataset " + fa.getDatasetName());
- }
- } catch (Exception exception) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to disconnect feed " + fa.getFeedName() + " from dataset "
- + fa.getDatasetName() + ". Encountered exception " + exception);
+ for (FeedConnectionId connection : activeFeedConnections) {
+ FeedId feedId = connection.getFeedId();
+ if (feedId.getDataverse().equals(dataverseName)) {
+ disStmt = new DisconnectFeedStatement(dvId, new Identifier(feedId.getFeedName()), new Identifier(
+ connection.getDatasetName()));
+ try {
+ handleDisconnectFeedStatement(metadataProvider, disStmt, hcc);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Disconnected feed " + feedId.getFeedName() + " from dataset "
+ + connection.getDatasetName());
+ }
+ } catch (Exception exception) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to disconnect feed " + feedId.getFeedName() + " from dataset "
+ + connection.getDatasetName() + ". Encountered exception " + exception);
+ }
}
}
}
@@ -1258,7 +1316,7 @@
private void handleDatasetDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
DropStatement stmtDelete = (DropStatement) stmt;
- String dataverseName = getActiveDataverseName(stmtDelete.getDataverseName());
+ String dataverseName = getActiveDataverse(stmtDelete.getDataverseName());
String datasetName = stmtDelete.getDatasetName().getValue();
ProgressState progress = ProgressState.NO_PROGRESS;
@@ -1281,19 +1339,18 @@
}
}
+ Map<FeedConnectionId, Pair<JobSpecification, Boolean>> disconnectJobList = new HashMap<FeedConnectionId, Pair<JobSpecification, Boolean>>();
if (ds.getDatasetType() == DatasetType.INTERNAL) {
// prepare job spec(s) that would disconnect any active feeds involving the dataset.
- 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);
+ List<FeedConnectionId> feedConnections = FeedLifecycleListener.INSTANCE.getActiveFeedConnections(null);
+ if (feedConnections != null && !feedConnections.isEmpty()) {
+ for (FeedConnectionId connection : feedConnections) {
+ Pair<JobSpecification, Boolean> p = FeedOperations.buildDisconnectFeedJobSpec(metadataProvider,
+ connection);
+ disconnectJobList.put(connection, p);
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Disconnected feed " + fa.getFeedName() + " from dataset " + datasetName
- + " as dataset is being dropped");
+ LOGGER.info("Disconnecting feed " + connection.getFeedId().getFeedName() + " from dataset "
+ + datasetName + " as dataset is being dropped");
}
}
}
@@ -1322,8 +1379,8 @@
progress = ProgressState.ADDED_PENDINGOP_RECORD_TO_METADATA;
//# disconnect the feeds
- for (JobSpecification jobSpec : disconnectFeedJobSpecs) {
- runJob(hcc, jobSpec, true);
+ for (Pair<JobSpecification, Boolean> p : disconnectJobList.values()) {
+ runJob(hcc, p.first, true);
}
//#. run the jobs
@@ -1427,7 +1484,7 @@
IndexDropStatement stmtIndexDrop = (IndexDropStatement) stmt;
String datasetName = stmtIndexDrop.getDatasetName().getValue();
- String dataverseName = getActiveDataverseName(stmtIndexDrop.getDataverseName());
+ String dataverseName = getActiveDataverse(stmtIndexDrop.getDataverseName());
ProgressState progress = ProgressState.NO_PROGRESS;
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
@@ -1447,19 +1504,24 @@
+ dataverseName);
}
- List<FeedActivity> feedActivities = MetadataManager.INSTANCE.getActiveFeeds(mdTxnCtx, dataverseName,
- datasetName);
- if (feedActivities != null && !feedActivities.isEmpty()) {
+ List<FeedConnectionId> feedConnections = FeedLifecycleListener.INSTANCE.getActiveFeedConnections(null);
+ boolean resourceInUse = false;
+ if (feedConnections != null && !feedConnections.isEmpty()) {
StringBuilder builder = new StringBuilder();
-
- for (FeedActivity fa : feedActivities) {
- builder.append(fa + "\n");
+ for (FeedConnectionId connection : feedConnections) {
+ if (connection.getDatasetName().equals(datasetName)) {
+ resourceInUse = true;
+ builder.append(connection + "\n");
+ }
}
- throw new AsterixException("Dataset" + datasetName
- + " is currently being fed into by the following feeds " + "." + builder.toString()
- + "\nOperation not supported.");
+ if (resourceInUse) {
+ 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);
@@ -1620,7 +1682,7 @@
private void handleTypeDropStatement(AqlMetadataProvider metadataProvider, Statement stmt) throws Exception {
TypeDropStatement stmtTypeDrop = (TypeDropStatement) stmt;
- String dataverseName = getActiveDataverseName(stmtTypeDrop.getDataverseName());
+ String dataverseName = getActiveDataverse(stmtTypeDrop.getDataverseName());
String typeName = stmtTypeDrop.getTypeName().getValue();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -1725,7 +1787,7 @@
private void handleLoadStatement(AqlMetadataProvider metadataProvider, Statement stmt, IHyracksClientConnection hcc)
throws Exception {
LoadStatement loadStmt = (LoadStatement) stmt;
- String dataverseName = getActiveDataverseName(loadStmt.getDataverseName());
+ String dataverseName = getActiveDataverse(loadStmt.getDataverseName());
String datasetName = loadStmt.getDatasetName().getValue();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
@@ -1756,7 +1818,7 @@
IHyracksClientConnection hcc) throws Exception {
InsertStatement stmtInsert = (InsertStatement) stmt;
- String dataverseName = getActiveDataverseName(stmtInsert.getDataverseName());
+ String dataverseName = getActiveDataverse(stmtInsert.getDataverseName());
Query query = stmtInsert.getQuery();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
@@ -1792,7 +1854,7 @@
IHyracksClientConnection hcc) throws Exception {
DeleteStatement stmtDelete = (DeleteStatement) stmt;
- String dataverseName = getActiveDataverseName(stmtDelete.getDataverseName());
+ String dataverseName = getActiveDataverse(stmtDelete.getDataverseName());
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -1846,29 +1908,40 @@
IHyracksClientConnection hcc) throws Exception {
CreateFeedStatement cfs = (CreateFeedStatement) stmt;
- String dataverseName = getActiveDataverseName(cfs.getDataverseName());
+ String dataverseName = getActiveDataverse(cfs.getDataverseName());
String feedName = cfs.getFeedName().getValue();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
MetadataLockManager.INSTANCE.createFeedBegin(dataverseName, dataverseName + "." + feedName);
- String adapterName = null;
Feed feed = null;
try {
- adapterName = cfs.getAdapterName();
-
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 " + adapterName + " already exists.");
+ throw new AlgebricksException("A feed with this name " + feedName + " already exists.");
}
}
- feed = new Feed(dataverseName, feedName, adapterName, cfs.getAdapterConfiguration(),
- cfs.getAppliedFunction());
+ switch (stmt.getKind()) {
+ case CREATE_PRIMARY_FEED:
+ CreatePrimaryFeedStatement cpfs = (CreatePrimaryFeedStatement) stmt;
+ String adaptorName = cpfs.getAdaptorName();
+ feed = new PrimaryFeed(dataverseName, feedName, adaptorName, cpfs.getAdaptorConfiguration(),
+ cfs.getAppliedFunction());
+ break;
+ case CREATE_SECONDARY_FEED:
+ CreateSecondaryFeedStatement csfs = (CreateSecondaryFeedStatement) stmt;
+ feed = new SecondaryFeed(dataverseName, feedName, csfs.getSourceFeedName(),
+ csfs.getAppliedFunction());
+ break;
+ default:
+ throw new IllegalStateException();
+ }
+
MetadataManager.INSTANCE.addFeed(metadataProvider.getMetadataTxnContext(), feed);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
} catch (Exception e) {
@@ -1878,11 +1951,72 @@
MetadataLockManager.INSTANCE.createFeedEnd(dataverseName, dataverseName + "." + feedName);
}
}
+
+ private void handleCreateFeedPolicyStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ IHyracksClientConnection hcc) throws Exception {
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ String dataverse;
+ String policy;
+ FeedPolicy newPolicy = null;
+ CreateFeedPolicyStatement cfps = (CreateFeedPolicyStatement) stmt;
+ dataverse = getActiveDataverse(null);
+ policy = cfps.getPolicyName();
+ MetadataLockManager.INSTANCE.createFeedPolicyBegin(dataverse, dataverse + "." + policy);
+ try {
+ FeedPolicy feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(metadataProvider.getMetadataTxnContext(),
+ dataverse, policy);
+ if (feedPolicy != null) {
+ if (cfps.getIfNotExists()) {
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return;
+ } else {
+ throw new AlgebricksException("A policy with this name " + policy + " already exists.");
+ }
+ }
+ boolean extendingExisting = cfps.getSourcePolicyName() != null;
+ String description = cfps.getDescription() == null ? "" : cfps.getDescription();
+ if (extendingExisting) {
+ FeedPolicy sourceFeedPolicy = MetadataManager.INSTANCE.getFeedPolicy(
+ metadataProvider.getMetadataTxnContext(), dataverse, cfps.getSourcePolicyName());
+ if (sourceFeedPolicy == null) {
+ sourceFeedPolicy = MetadataManager.INSTANCE.getFeedPolicy(metadataProvider.getMetadataTxnContext(),
+ MetadataConstants.METADATA_DATAVERSE_NAME, cfps.getSourcePolicyName());
+ if (sourceFeedPolicy == null) {
+ throw new AlgebricksException("Unknown policy " + cfps.getSourcePolicyName());
+ }
+ }
+ Map<String, String> policyProperties = sourceFeedPolicy.getProperties();
+ policyProperties.putAll(cfps.getProperties());
+ newPolicy = new FeedPolicy(dataverse, policy, description, policyProperties);
+ } else {
+ Properties prop = new Properties();
+ try {
+ InputStream stream = new FileInputStream(cfps.getSourcePolicyFile());
+ prop.load(stream);
+ } catch (Exception e) {
+ throw new AlgebricksException("Unable to read policy file" + cfps.getSourcePolicyFile());
+ }
+ Map<String, String> policyProperties = new HashMap<String, String>();
+ for (Entry<Object, Object> entry : prop.entrySet()) {
+ policyProperties.put((String) entry.getKey(), (String) entry.getValue());
+ }
+ newPolicy = new FeedPolicy(dataverse, policy, description, policyProperties);
+ }
+ MetadataManager.INSTANCE.addFeedPolicy(mdTxnCtx, newPolicy);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e) {
+ abort(e, e, mdTxnCtx);
+ throw e;
+ } finally {
+ MetadataLockManager.INSTANCE.createFeedPolicyEnd(dataverse, dataverse + "." + policy);
+ }
+ }
private void handleDropFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
FeedDropStatement stmtFeedDrop = (FeedDropStatement) stmt;
- String dataverseName = getActiveDataverseName(stmtFeedDrop.getDataverseName());
+ String dataverseName = getActiveDataverse(stmtFeedDrop.getDataverseName());
String feedName = stmtFeedDrop.getFeedName().getValue();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -1896,27 +2030,25 @@
}
}
- List<FeedActivity> feedActivities;
- try {
- feedActivities = MetadataManager.INSTANCE.getConnectFeedActivitiesForFeed(mdTxnCtx, dataverseName,
- feedName);
+ FeedId feedId = new FeedId(dataverseName, feedName);
+ List<FeedConnectionId> activeConnections = FeedLifecycleListener.INSTANCE.getActiveFeedConnections(feedId);
+ if (activeConnections != null && !activeConnections.isEmpty()) {
+ StringBuilder builder = new StringBuilder();
+ for (FeedConnectionId connectionId : activeConnections) {
+ builder.append(connectionId.getDatasetName() + "\n");
+ }
+
+ throw new AlgebricksException("Feed " + feedId
+ + " is currently active and connected to the following dataset(s) \n" + builder.toString());
+ } else {
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);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Removed feed " + feedId);
}
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
- for (JobSpecification spec : jobSpecs) {
- runJob(hcc, spec, true);
- }
} catch (Exception e) {
abort(e, e, mdTxnCtx);
@@ -1925,78 +2057,106 @@
MetadataLockManager.INSTANCE.dropFeedEnd(dataverseName, dataverseName + "." + feedName);
}
}
+
+ private void handleDropFeedPolicyStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ IHyracksClientConnection hcc) throws Exception {
+
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ FeedPolicyDropStatement stmtFeedPolicyDrop = (FeedPolicyDropStatement) stmt;
+ String dataverseName = getActiveDataverse(stmtFeedPolicyDrop.getDataverseName());
+ String policyName = stmtFeedPolicyDrop.getPolicyName().getValue();
+ MetadataLockManager.INSTANCE.dropFeedPolicyBegin(dataverseName, dataverseName + "." + policyName);
+
+ try {
+ FeedPolicy feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(mdTxnCtx, dataverseName, policyName);
+ if (feedPolicy == null) {
+ if (!stmtFeedPolicyDrop.getIfExists()) {
+ throw new AlgebricksException("Unknown policy " + policyName + " in dataverse " + dataverseName);
+ }
+ }
+ MetadataManager.INSTANCE.dropFeedPolicy(mdTxnCtx, dataverseName, policyName);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e) {
+ abort(e, e, mdTxnCtx);
+ throw e;
+ } finally {
+ MetadataLockManager.INSTANCE.dropFeedPolicyEnd(dataverseName, dataverseName + "." + policyName);
+ }
+ }
+
private void handleConnectFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
ConnectFeedStatement cfs = (ConnectFeedStatement) stmt;
- String dataverseName = getActiveDataverseName(cfs.getDataverseName());
+ String dataverseName = getActiveDataverse(cfs.getDataverseName());
String feedName = cfs.getFeedName();
String datasetName = cfs.getDatasetName().getValue();
- MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
+
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ boolean readLatchAcquired = true;
+ boolean subscriberRegistered = false;
+ IFeedLifecycleEventSubscriber eventSubscriber = new FeedLifecycleEventSubscriber();
+ FeedConnectionId feedConnId = null;
+
MetadataLockManager.INSTANCE.connectFeedBegin(dataverseName, dataverseName + "." + datasetName, dataverseName
+ "." + feedName);
- boolean readLatchAcquired = true;
try {
-
metadataProvider.setWriteTransaction(true);
CompiledConnectFeedStatement cbfs = new CompiledConnectFeedStatement(dataverseName, cfs.getFeedName(), cfs
.getDatasetName().getValue(), cfs.getPolicy(), cfs.getQuery(), cfs.getVarCounter());
- Dataset dataset = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(),
- dataverseName, cfs.getDatasetName().getValue());
- if (dataset == null) {
- throw new AsterixException("Unknown target dataset :" + cfs.getDatasetName().getValue());
+ FeedUtil.validateIfDatasetExists(dataverseName, cfs.getDatasetName().getValue(),
+ metadataProvider.getMetadataTxnContext());
+
+ Feed feed = FeedUtil.validateIfFeedExists(dataverseName, cfs.getFeedName(),
+ metadataProvider.getMetadataTxnContext());
+
+ feedConnId = new FeedConnectionId(dataverseName, cfs.getFeedName(), cfs.getDatasetName().getValue());
+
+ if (FeedLifecycleListener.INSTANCE.isFeedConnectionActive(feedConnId)) {
+ throw new AsterixException("Feed " + cfs.getFeedName() + " is already connected to 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);
- }
+ FeedPolicy feedPolicy = FeedUtil.validateIfPolicyExists(dataverseName, cbfs.getPolicyName(), mdTxnCtx);
- Feed feed = MetadataManager.INSTANCE.getFeed(metadataProvider.getMetadataTxnContext(), dataverseName,
- cfs.getFeedName());
- if (feed == null) {
- throw new AsterixException("Unknown source feed: " + cfs.getFeedName());
- }
+ // All Metadata checks have passed. Feed connect request is valid. //
- 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");
- }
+ FeedPolicyAccessor policyAccessor = new FeedPolicyAccessor(feedPolicy.getProperties());
+ Pair<FeedConnectionRequest, Boolean> p = getFeedConnectionRequest(dataverseName, feed,
+ cbfs.getDatasetName(), feedPolicy, mdTxnCtx);
+ FeedConnectionRequest connectionRequest = p.first;
+ boolean createFeedIntakeJob = p.second;
- 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());
+ FeedLifecycleListener.INSTANCE.registerFeedEventSubscriber(feedConnId, eventSubscriber);
+ subscriberRegistered = true;
+ if (createFeedIntakeJob) {
+ FeedId feedId = connectionRequest.getFeedJointKey().getFeedId();
+ PrimaryFeed primaryFeed = (PrimaryFeed) MetadataManager.INSTANCE.getFeed(mdTxnCtx,
+ feedId.getDataverse(), feedId.getFeedName());
+ Pair<JobSpecification, IFeedAdapterFactory> pair = FeedOperations.buildFeedIntakeJobSpec(primaryFeed,
+ metadataProvider, policyAccessor);
+ runJob(hcc, pair.first, false);
+ IFeedAdapterFactory adapterFactory = pair.second;
+ if (adapterFactory.isRecordTrackingEnabled()) {
+ FeedLifecycleListener.INSTANCE.registerFeedIntakeProgressTracker(feedConnId,
+ adapterFactory.createIntakeProgressTracker());
}
+ eventSubscriber.assertEvent(FeedLifecycleEvent.FEED_INTAKE_STARTED);
}
-
- 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;
+ readLatchAcquired = false;
+ eventSubscriber.assertEvent(FeedLifecycleEvent.FEED_COLLECT_STARTED);
+ if (Boolean.valueOf(metadataProvider.getConfig().get(ConnectFeedStatement.WAIT_FOR_COMPLETION))) {
+ eventSubscriber.assertEvent(FeedLifecycleEvent.FEED_ENDED); // blocking call
+ }
String waitForCompletionParam = metadataProvider.getConfig().get(ConnectFeedStatement.WAIT_FOR_COMPLETION);
boolean waitForCompletion = waitForCompletionParam == null ? false : Boolean
.valueOf(waitForCompletionParam);
@@ -2005,7 +2165,6 @@
dataverseName + "." + feedName);
readLatchAcquired = false;
}
- runJob(hcc, newJobSpec, waitForCompletion);
} catch (Exception e) {
if (bActiveTxn) {
abort(e, e, mdTxnCtx);
@@ -2016,18 +2175,127 @@
MetadataLockManager.INSTANCE.connectFeedEnd(dataverseName, dataverseName + "." + datasetName,
dataverseName + "." + feedName);
}
+ if (subscriberRegistered) {
+ FeedLifecycleListener.INSTANCE.deregisterFeedEventSubscriber(feedConnId, eventSubscriber);
+ }
}
}
+
+ /**
+ * Generates a subscription request corresponding to a connect feed request. In addition, provides a boolean
+ * flag indicating if feed intake job needs to be started (source primary feed not found to be active).
+ *
+ * @param dataverse
+ * @param feed
+ * @param dataset
+ * @param feedPolicy
+ * @param mdTxnCtx
+ * @return
+ * @throws MetadataException
+ */
+ private Pair<FeedConnectionRequest, Boolean> getFeedConnectionRequest(String dataverse, Feed feed, String dataset,
+ FeedPolicy feedPolicy, MetadataTransactionContext mdTxnCtx) throws MetadataException {
+ IFeedJoint sourceFeedJoint = null;
+ FeedConnectionRequest request = null;
+ List<String> functionsToApply = new ArrayList<String>();
+ boolean needIntakeJob = false;
+ List<IFeedJoint> jointsToRegister = new ArrayList<IFeedJoint>();
+ FeedConnectionId connectionId = new FeedConnectionId(feed.getFeedId(), dataset);
+ ConnectionLocation connectionLocation = null;
+ FeedJointKey feedJointKey = getFeedJointKey(feed, mdTxnCtx);
+ boolean isFeedJointAvailable = FeedLifecycleListener.INSTANCE.isFeedJointAvailable(feedJointKey);
+ if (!isFeedJointAvailable) {
+ sourceFeedJoint = FeedLifecycleListener.INSTANCE.getAvailableFeedJoint(feedJointKey);
+ if (sourceFeedJoint == null) { // the feed is currently not being ingested, i.e., it is unavailable.
+ connectionLocation = ConnectionLocation.SOURCE_FEED_INTAKE_STAGE;
+ FeedId sourceFeedId = feedJointKey.getFeedId(); // the root/primary feedId
+ Feed primaryFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, dataverse, sourceFeedId.getFeedName());
+ FeedJointKey intakeFeedJointKey = new FeedJointKey(sourceFeedId, new ArrayList<String>());
+ sourceFeedJoint = new FeedJoint(intakeFeedJointKey, primaryFeed.getFeedId(), connectionLocation,
+ FeedJointType.INTAKE, connectionId);
+ jointsToRegister.add(sourceFeedJoint);
+ needIntakeJob = true;
+ } else {
+ connectionLocation = sourceFeedJoint.getConnectionLocation();
+ }
+
+ String[] functions = feedJointKey.getStringRep()
+ .substring(sourceFeedJoint.getFeedJointKey().getStringRep().length()).trim().split(":");
+ for (String f : functions) {
+ if (f.trim().length() > 0) {
+ functionsToApply.add(f);
+ }
+ }
+ // register the compute feed point that represents the final output from the collection of
+ // functions that will be applied.
+ if (!functionsToApply.isEmpty()) {
+ FeedJointKey computeFeedJointKey = new FeedJointKey(feed.getFeedId(), functionsToApply);
+ IFeedJoint computeFeedJoint = new FeedJoint(computeFeedJointKey, feed.getFeedId(),
+ ConnectionLocation.SOURCE_FEED_COMPUTE_STAGE, FeedJointType.COMPUTE, connectionId);
+ jointsToRegister.add(computeFeedJoint);
+ }
+ for (IFeedJoint joint : jointsToRegister) {
+ FeedLifecycleListener.INSTANCE.registerFeedJoint(joint);
+ }
+ } else {
+ sourceFeedJoint = FeedLifecycleListener.INSTANCE.getFeedJoint(feedJointKey);
+ connectionLocation = sourceFeedJoint.getConnectionLocation();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Feed joint " + sourceFeedJoint + " is available! need not apply any further computation");
+ }
+ }
+
+ request = new FeedConnectionRequest(sourceFeedJoint.getFeedJointKey(), connectionLocation, functionsToApply,
+ dataset, feedPolicy.getPolicyName(), feedPolicy.getProperties(), feed.getFeedId());
+
+ sourceFeedJoint.addConnectionRequest(request);
+ return new Pair<FeedConnectionRequest, Boolean>(request, needIntakeJob);
+ }
+
+ /*
+ * Gets the feed joint corresponding to the feed definition. Tuples constituting the feed are
+ * available at this feed joint.
+ */
+ private FeedJointKey getFeedJointKey(Feed feed, MetadataTransactionContext ctx) throws MetadataException {
+ Feed sourceFeed = feed;
+ List<String> appliedFunctions = new ArrayList<String>();
+ while (sourceFeed.getFeedType().equals(FeedType.SECONDARY)) {
+ if (sourceFeed.getAppliedFunction() != null) {
+ appliedFunctions.add(0, sourceFeed.getAppliedFunction().getName());
+ }
+ Feed parentFeed = MetadataManager.INSTANCE.getFeed(ctx, feed.getDataverseName(),
+ ((SecondaryFeed) sourceFeed).getSourceFeedName());
+ sourceFeed = parentFeed;
+ }
+
+ if (sourceFeed.getAppliedFunction() != null) {
+ appliedFunctions.add(0, sourceFeed.getAppliedFunction().getName());
+ }
+
+ return new FeedJointKey(sourceFeed.getFeedId(), appliedFunctions);
+ }
+
private void handleDisconnectFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
DisconnectFeedStatement cfs = (DisconnectFeedStatement) stmt;
- String dataverseName = getActiveDataverseName(cfs.getDataverseName());
+ String dataverseName = getActiveDataverse(cfs.getDataverseName());
String datasetName = cfs.getDatasetName().getValue();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+ FeedUtil.validateIfDatasetExists(dataverseName, cfs.getDatasetName().getValue(), mdTxnCtx);
+ Feed feed = FeedUtil.validateIfFeedExists(dataverseName, cfs.getFeedName().getValue(), mdTxnCtx);
+
+ FeedConnectionId connectionId = new FeedConnectionId(feed.getFeedId(), cfs.getDatasetName().getValue());
+ boolean isFeedConnectionActive = FeedLifecycleListener.INSTANCE.isFeedConnectionActive(connectionId);
+ if (!isFeedConnectionActive) {
+ throw new AsterixException("Feed " + feed.getFeedId().getFeedName() + " is currently not connected to "
+ + cfs.getDatasetName().getValue() + ". Invalid operation!");
+ }
+
MetadataLockManager.INSTANCE.disconnectFeedBegin(dataverseName, dataverseName + "." + datasetName,
dataverseName + "." + cfs.getFeedName());
try {
@@ -2037,32 +2305,19 @@
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");
- }
-
- JobSpecification jobSpec = FeedOperations.buildDisconnectFeedJobSpec(dataverseName, cfs.getFeedName()
- .getValue(), cfs.getDatasetName().getValue(), metadataProvider, feedActivity);
-
+ Pair<JobSpecification, Boolean> specDisconnectType = FeedOperations.buildDisconnectFeedJobSpec(
+ metadataProvider, connectionId);
+ JobSpecification jobSpec = specDisconnectType.first;
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
runJob(hcc, jobSpec, true);
+
+ if (!specDisconnectType.second) {
+ CentralFeedManager.getInstance().getFeedLoadManager().removeFeedActivity(connectionId);
+ FeedLifecycleListener.INSTANCE.reportPartialDisconnection(connectionId);
+ }
+
} catch (Exception e) {
if (bActiveTxn) {
abort(e, e, mdTxnCtx);
@@ -2073,11 +2328,64 @@
dataverseName + "." + cfs.getFeedName());
}
}
+
+ private void handleSubscribeFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
+ IHyracksClientConnection hcc) throws Exception {
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Subscriber Feed Statement :" + stmt);
+ }
+
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ boolean bActiveTxn = true;
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ metadataProvider.setWriteTransaction(true);
+ SubscribeFeedStatement bfs = (SubscribeFeedStatement) stmt;
+ bfs.initialize(metadataProvider.getMetadataTxnContext());
+
+ CompiledSubscribeFeedStatement csfs = new CompiledSubscribeFeedStatement(bfs.getSubscriptionRequest(),
+ bfs.getQuery(), bfs.getVarCounter());
+ metadataProvider.getConfig().put(FunctionUtils.IMPORT_PRIVATE_FUNCTIONS, "" + Boolean.TRUE);
+ metadataProvider.getConfig().put(FeedActivityDetails.FEED_POLICY_NAME, "" + bfs.getPolicy());
+ metadataProvider.getConfig().put(FeedActivityDetails.COLLECT_LOCATIONS,
+ StringUtils.join(bfs.getLocations(), ','));
+
+ JobSpecification compiled = rewriteCompileQuery(metadataProvider, bfs.getQuery(), csfs);
+ FeedConnectionId feedConnectionId = new FeedConnectionId(bfs.getSubscriptionRequest().getReceivingFeedId(), bfs
+ .getSubscriptionRequest().getTargetDataset());
+ String dataverse = feedConnectionId.getFeedId().getDataverse();
+ String dataset = feedConnectionId.getDatasetName();
+ MetadataLockManager.INSTANCE.subscribeFeedBegin(dataverse, dataverse + "." + dataset, dataverse + "."
+ + feedConnectionId.getFeedId().getFeedName());
+
+ try {
+
+
+ JobSpecification alteredJobSpec = FeedUtil.alterJobSpecificationForFeed(compiled, feedConnectionId, bfs
+ .getSubscriptionRequest().getPolicyParameters());
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ bActiveTxn = false;
+
+ if (compiled != null) {
+ runJob(hcc, alteredJobSpec, false);
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ if (bActiveTxn) {
+ abort(e, e, mdTxnCtx);
+ }
+ throw e;
+ } finally {
+ MetadataLockManager.INSTANCE.subscribeFeedEnd(dataverse, dataverse + "." + dataset, dataverse + "."
+ + feedConnectionId.getFeedId().getFeedName());
+ }
+ }
private void handleCompactStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
CompactStatement compactStatement = (CompactStatement) stmt;
- String dataverseName = getActiveDataverseName(compactStatement.getDataverseName());
+ String dataverseName = getActiveDataverse(compactStatement.getDataverseName());
String datasetName = compactStatement.getDatasetName().getValue();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
@@ -2261,7 +2569,7 @@
private void handleExternalDatasetRefreshStatement(AqlMetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc) throws Exception {
RefreshExternalDatasetStatement stmtRefresh = (RefreshExternalDatasetStatement) stmt;
- String dataverseName = getActiveDataverseName(stmtRefresh.getDataverseName());
+ String dataverseName = getActiveDataverse(stmtRefresh.getDataverseName());
String datasetName = stmtRefresh.getDatasetName().getValue();
ExternalDatasetTransactionState transactionState = ExternalDatasetTransactionState.COMMIT;
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
@@ -2503,8 +2811,8 @@
RunStatement pregelixStmt = (RunStatement) stmt;
boolean bActiveTxn = true;
- String dataverseNameFrom = getActiveDataverseName(pregelixStmt.getDataverseNameFrom());
- String dataverseNameTo = getActiveDataverseName(pregelixStmt.getDataverseNameTo());
+ String dataverseNameFrom = getActiveDataverse(pregelixStmt.getDataverseNameFrom());
+ String dataverseNameTo = getActiveDataverse(pregelixStmt.getDataverseNameTo());
String datasetNameFrom = pregelixStmt.getDatasetNameFrom().getValue();
String datasetNameTo = pregelixStmt.getDatasetNameTo().getValue();
@@ -2745,7 +3053,7 @@
throw new AlgebricksException("dataverse not specified");
}
- private String getActiveDataverseName(Identifier dataverse) throws AlgebricksException {
+ private String getActiveDataverse(Identifier dataverse) throws AlgebricksException {
return getActiveDataverseName(dataverse != null ? dataverse.getValue() : null);
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/CentralFeedManager.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/CentralFeedManager.java
new file mode 100644
index 0000000..359da16
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/CentralFeedManager.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2009-2014 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.feeds;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringReader;
+import java.util.List;
+
+import edu.uci.ics.asterix.api.common.SessionConfig;
+import edu.uci.ics.asterix.api.common.SessionConfig.OutputFormat;
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.parser.AQLParser;
+import edu.uci.ics.asterix.aql.translator.AqlTranslator;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.api.ICentralFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLoadManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedTrackingManager;
+import edu.uci.ics.asterix.metadata.feeds.SocketMessageListener;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class CentralFeedManager implements ICentralFeedManager {
+
+ private static final ICentralFeedManager centralFeedManager = new CentralFeedManager();
+
+ public static ICentralFeedManager getInstance() {
+ return centralFeedManager;
+ }
+
+ private final int port;
+ private final IFeedLoadManager feedLoadManager;
+ private final IFeedTrackingManager feedTrackingManager;
+ private final SocketMessageListener messageListener;
+
+ private CentralFeedManager() {
+ this.port = AsterixAppContextInfo.getInstance().getFeedProperties().getFeedCentralManagerPort();
+ this.feedLoadManager = new FeedLoadManager();
+ this.feedTrackingManager = new FeedTrackingManager();
+ this.messageListener = new SocketMessageListener(port, new FeedMessageReceiver(this));
+ }
+
+ @Override
+ public void start() throws AsterixException {
+ messageListener.start();
+ }
+
+ @Override
+ public void stop() throws AsterixException, IOException {
+ messageListener.stop();
+ }
+
+ public static JobId runJob(JobSpecification spec, boolean waitForCompletion) throws Exception {
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ JobId jobId = hcc.startJob(spec);
+ if (waitForCompletion) {
+ hcc.waitForCompletion(jobId);
+ }
+ return jobId;
+ }
+
+ @Override
+ public IFeedLoadManager getFeedLoadManager() {
+ return feedLoadManager;
+ }
+
+ @Override
+ public IFeedTrackingManager getFeedTrackingManager() {
+ return feedTrackingManager;
+ }
+
+ public static class AQLExecutor {
+
+ private static final PrintWriter out = new PrintWriter(System.out, true);
+
+ public static void executeAQL(String aql) throws Exception {
+ AQLParser parser = new AQLParser(new StringReader(aql));
+ List<Statement> statements;
+ statements = parser.Statement();
+ SessionConfig pc = new SessionConfig(out, OutputFormat.ADM);
+ AqlTranslator translator = new AqlTranslator(statements, pc);
+ translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null, AqlTranslator.ResultDelivery.SYNC);
+ }
+ }
+
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedCollectInfo.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedCollectInfo.java
new file mode 100644
index 0000000..db3ce77
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedCollectInfo.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.asterix.feeds;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class FeedCollectInfo extends FeedInfo {
+ public FeedId sourceFeedId;
+ public FeedConnectionId feedConnectionId;
+ public List<String> collectLocations = new ArrayList<String>();
+ public List<String> computeLocations = new ArrayList<String>();
+ public List<String> storageLocations = new ArrayList<String>();
+ public Map<String, String> feedPolicy;
+ public String superFeedManagerHost;
+ public int superFeedManagerPort;
+ public boolean fullyConnected;
+
+ public FeedCollectInfo(FeedId sourceFeedId, FeedConnectionId feedConnectionId, JobSpecification jobSpec,
+ JobId jobId, Map<String, String> feedPolicy) {
+ super(jobSpec, jobId, FeedInfoType.COLLECT);
+ this.sourceFeedId = sourceFeedId;
+ this.feedConnectionId = feedConnectionId;
+ this.feedPolicy = feedPolicy;
+ this.fullyConnected = true;
+ }
+
+ @Override
+ public String toString() {
+ return FeedInfoType.COLLECT + "[" + feedConnectionId + "]";
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedInfo.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedInfo.java
new file mode 100644
index 0000000..c81b73f
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedInfo.java
@@ -0,0 +1,35 @@
+package edu.uci.ics.asterix.feeds;
+
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobInfo;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class FeedInfo {
+ public JobSpecification jobSpec;
+ public JobInfo jobInfo;
+ public JobId jobId;
+ public FeedInfoType infoType;
+ public State state;
+
+ public enum State {
+ ACTIVE,
+ INACTIVE
+ }
+
+ public enum FeedInfoType {
+ INTAKE,
+ COLLECT
+ }
+
+ public FeedInfo(JobSpecification jobSpec, JobId jobId, FeedInfoType infoType) {
+ this.jobSpec = jobSpec;
+ this.jobId = jobId;
+ this.infoType = infoType;
+ this.state = State.INACTIVE;
+ }
+
+ @Override
+ public String toString() {
+ return " job id " + jobId;
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedJobNotificationHandler.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedJobNotificationHandler.java
new file mode 100644
index 0000000..a320faa
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedJobNotificationHandler.java
@@ -0,0 +1,739 @@
+/*
+ * 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.feeds;
+
+import java.rmi.RemoteException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.commons.lang3.StringUtils;
+
+import edu.uci.ics.asterix.api.common.FeedWorkCollection.SubscribeFeedWork;
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.feeds.FeedActivity;
+import edu.uci.ics.asterix.common.feeds.FeedConnectJobInfo;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionRequest;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedIntakeInfo;
+import edu.uci.ics.asterix.common.feeds.FeedJobInfo;
+import edu.uci.ics.asterix.common.feeds.FeedJobInfo.FeedJobState;
+import edu.uci.ics.asterix.common.feeds.FeedJobInfo.JobType;
+import edu.uci.ics.asterix.common.feeds.FeedJointKey;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.feeds.api.IFeedJoint;
+import edu.uci.ics.asterix.common.feeds.api.IFeedJoint.State;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleEventSubscriber;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleEventSubscriber.FeedLifecycleEvent;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
+import edu.uci.ics.asterix.common.feeds.message.StorageReportFeedMessage;
+import edu.uci.ics.asterix.feeds.FeedLifecycleListener.Message;
+import edu.uci.ics.asterix.metadata.feeds.BuiltinFeedPolicies;
+import edu.uci.ics.asterix.metadata.feeds.FeedCollectOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.feeds.FeedIntakeOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.feeds.FeedMetaOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.feeds.FeedWorkManager;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+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.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+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.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
+
+public class FeedJobNotificationHandler implements Runnable {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedJobNotificationHandler.class.getName());
+
+ private final LinkedBlockingQueue<Message> inbox;
+ private final Map<FeedConnectionId, List<IFeedLifecycleEventSubscriber>> eventSubscribers;
+
+ private final Map<JobId, FeedJobInfo> jobInfos;
+ private final Map<FeedId, FeedIntakeInfo> intakeJobInfos;
+ private final Map<FeedConnectionId, FeedConnectJobInfo> connectJobInfos;
+ private final Map<FeedId, List<IFeedJoint>> feedPipeline;
+ private final Map<FeedConnectionId, Pair<IIntakeProgressTracker, Long>> feedIntakeProgressTrackers;
+
+ public FeedJobNotificationHandler(LinkedBlockingQueue<Message> inbox) {
+ this.inbox = inbox;
+ this.jobInfos = new HashMap<JobId, FeedJobInfo>();
+ this.intakeJobInfos = new HashMap<FeedId, FeedIntakeInfo>();
+ this.connectJobInfos = new HashMap<FeedConnectionId, FeedConnectJobInfo>();
+ this.feedPipeline = new HashMap<FeedId, List<IFeedJoint>>();
+ this.eventSubscribers = new HashMap<FeedConnectionId, List<IFeedLifecycleEventSubscriber>>();
+ this.feedIntakeProgressTrackers = new HashMap<FeedConnectionId, Pair<IIntakeProgressTracker, Long>>();
+ }
+
+ @Override
+ public void run() {
+ Message mesg;
+ while (true) {
+ try {
+ mesg = inbox.take();
+ switch (mesg.messageKind) {
+ case JOB_START:
+ handleJobStartMessage(mesg);
+ break;
+ case JOB_FINISH:
+ handleJobFinishMessage(mesg);
+ break;
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+
+ }
+ }
+
+ public void registerFeedIntakeProgressTracker(FeedConnectionId connectionId,
+ IIntakeProgressTracker feedIntakeProgressTracker) {
+ if (feedIntakeProgressTrackers.get(connectionId) == null) {
+ this.feedIntakeProgressTrackers.put(connectionId, new Pair<IIntakeProgressTracker, Long>(
+ feedIntakeProgressTracker, 0L));
+ } else {
+ throw new IllegalStateException(" Progress tracker for connection " + connectionId
+ + " is alreader registered");
+ }
+ }
+
+ public void deregisterFeedIntakeProgressTracker(FeedConnectionId connectionId) {
+ this.feedIntakeProgressTrackers.remove(connectionId);
+ }
+
+ public void updateTrackingInformation(StorageReportFeedMessage srm) {
+ Pair<IIntakeProgressTracker, Long> p = feedIntakeProgressTrackers.get(srm.getConnectionId());
+ if (p != null && p.second < srm.getLastPersistedTupleIntakeTimestamp()) {
+ p.second = srm.getLastPersistedTupleIntakeTimestamp();
+ p.first.notifyIngestedTupleTimestamp(p.second);
+ }
+ }
+
+ public Collection<FeedIntakeInfo> getFeedIntakeInfos() {
+ return intakeJobInfos.values();
+ }
+
+ public Collection<FeedConnectJobInfo> getFeedConnectInfos() {
+ return connectJobInfos.values();
+ }
+
+ public void registerFeedJoint(IFeedJoint feedJoint) {
+ List<IFeedJoint> feedJointsOnPipeline = feedPipeline.get(feedJoint.getOwnerFeedId());
+ if (feedJointsOnPipeline == null) {
+ feedJointsOnPipeline = new ArrayList<IFeedJoint>();
+ feedPipeline.put(feedJoint.getOwnerFeedId(), feedJointsOnPipeline);
+ feedJointsOnPipeline.add(feedJoint);
+ } else {
+ if (!feedJointsOnPipeline.contains(feedJoint)) {
+ feedJointsOnPipeline.add(feedJoint);
+ } else {
+ throw new IllegalArgumentException("Feed joint " + feedJoint + " already registered");
+ }
+ }
+ }
+
+ public void registerFeedIntakeJob(FeedId feedId, JobId jobId, JobSpecification jobSpec) throws HyracksDataException {
+ if (jobInfos.get(jobId) != null) {
+ throw new IllegalStateException("Feed job already registered");
+ }
+
+ List<IFeedJoint> joints = feedPipeline.get(feedId);
+ IFeedJoint intakeJoint = null;
+ for (IFeedJoint joint : joints) {
+ if (joint.getType().equals(IFeedJoint.FeedJointType.INTAKE)) {
+ intakeJoint = joint;
+ break;
+ }
+ }
+
+ if (intakeJoint != null) {
+ FeedIntakeInfo intakeJobInfo = new FeedIntakeInfo(jobId, FeedJobState.CREATED, FeedJobInfo.JobType.INTAKE,
+ feedId, intakeJoint, jobSpec);
+ intakeJobInfos.put(feedId, intakeJobInfo);
+ jobInfos.put(jobId, intakeJobInfo);
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Registered feed intake [" + jobId + "]" + " for feed " + feedId);
+ }
+ } else {
+ throw new HyracksDataException("Could not register feed intake job [" + jobId + "]" + " for feed "
+ + feedId);
+ }
+ }
+
+ public void registerFeedCollectionJob(FeedId sourceFeedId, FeedConnectionId connectionId, JobId jobId,
+ JobSpecification jobSpec, Map<String, String> feedPolicy) {
+ if (jobInfos.get(jobId) != null) {
+ throw new IllegalStateException("Feed job already registered");
+ }
+
+ List<IFeedJoint> feedJoints = feedPipeline.get(sourceFeedId);
+ FeedConnectionId cid = null;
+ IFeedJoint sourceFeedJoint = null;
+ for (IFeedJoint joint : feedJoints) {
+ cid = joint.getReceiver(connectionId);
+ if (cid != null) {
+ sourceFeedJoint = joint;
+ break;
+ }
+ }
+
+ if (cid != null) {
+ FeedConnectJobInfo cInfo = new FeedConnectJobInfo(jobId, FeedJobState.CREATED, connectionId,
+ sourceFeedJoint, null, jobSpec, feedPolicy);
+ jobInfos.put(jobId, cInfo);
+ connectJobInfos.put(connectionId, cInfo);
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Registered feed connection [" + jobId + "]" + " for feed " + connectionId);
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Could not register feed collection job [" + jobId + "]" + " for feed connection "
+ + connectionId);
+ }
+ }
+
+ }
+
+ public void deregisterFeedIntakeJob(JobId jobId) {
+ if (jobInfos.get(jobId) == null) {
+ throw new IllegalStateException(" Feed Intake job not registered ");
+ }
+
+ FeedIntakeInfo info = (FeedIntakeInfo) jobInfos.get(jobId);
+ jobInfos.remove(jobId);
+ intakeJobInfos.remove(info.getFeedId());
+
+ if (!info.getState().equals(FeedJobState.UNDER_RECOVERY)) {
+ List<IFeedJoint> joints = feedPipeline.get(info.getFeedId());
+ joints.remove(info.getIntakeFeedJoint());
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Deregistered feed intake job [" + jobId + "]");
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Not removing feed joint as intake job is in " + FeedJobState.UNDER_RECOVERY + " state.");
+ }
+ }
+
+ }
+
+ private void handleJobStartMessage(Message message) throws Exception {
+ FeedJobInfo jobInfo = jobInfos.get(message.jobId);
+ switch (jobInfo.getJobType()) {
+ case INTAKE:
+ handleIntakeJobStartMessage((FeedIntakeInfo) jobInfo);
+ break;
+ case FEED_CONNECT:
+ handleCollectJobStartMessage((FeedConnectJobInfo) jobInfo);
+ break;
+ }
+
+ }
+
+ private void handleJobFinishMessage(Message message) throws Exception {
+ FeedJobInfo jobInfo = jobInfos.get(message.jobId);
+ switch (jobInfo.getJobType()) {
+ case INTAKE:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Intake Job finished for feed intake " + jobInfo.getJobId());
+ }
+ handleFeedIntakeJobFinishMessage((FeedIntakeInfo) jobInfo, message);
+ break;
+ case FEED_CONNECT:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Collect Job finished for " + (FeedConnectJobInfo) jobInfo);
+ }
+ handleFeedCollectJobFinishMessage((FeedConnectJobInfo) jobInfo);
+ break;
+ }
+
+ }
+
+ private synchronized void handleIntakeJobStartMessage(FeedIntakeInfo intakeJobInfo) throws Exception {
+ List<OperatorDescriptorId> intakeOperatorIds = new ArrayList<OperatorDescriptorId>();
+ Map<OperatorDescriptorId, IOperatorDescriptor> operators = intakeJobInfo.getSpec().getOperatorMap();
+ for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operators.entrySet()) {
+ IOperatorDescriptor opDesc = entry.getValue();
+ if (opDesc instanceof FeedIntakeOperatorDescriptor) {
+ intakeOperatorIds.add(opDesc.getOperatorId());
+ }
+ }
+
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ JobInfo info = hcc.getJobInfo(intakeJobInfo.getJobId());
+ List<String> intakeLocations = new ArrayList<String>();
+ for (OperatorDescriptorId intakeOperatorId : intakeOperatorIds) {
+ Map<Integer, String> operatorLocations = info.getOperatorLocations().get(intakeOperatorId);
+ int nOperatorInstances = operatorLocations.size();
+ for (int i = 0; i < nOperatorInstances; i++) {
+ intakeLocations.add(operatorLocations.get(i));
+ }
+ }
+ // intakeLocations is an ordered list; element at position i corresponds to location of i'th instance of operator
+ intakeJobInfo.setIntakeLocation(intakeLocations);
+ intakeJobInfo.getIntakeFeedJoint().setState(State.ACTIVE);
+ intakeJobInfo.setState(FeedJobState.ACTIVE);
+
+ // notify event listeners
+ notifyFeedEventSubscribers(intakeJobInfo, FeedLifecycleEvent.FEED_INTAKE_STARTED);
+ }
+
+ private void handleCollectJobStartMessage(FeedConnectJobInfo cInfo) throws RemoteException, ACIDException {
+ // set locations of feed sub-operations (intake, compute, store)
+ setLocations(cInfo);
+
+ // activate joints
+ List<IFeedJoint> joints = feedPipeline.get(cInfo.getConnectionId().getFeedId());
+ for (IFeedJoint joint : joints) {
+ if (joint.getProvider().equals(cInfo.getConnectionId())) {
+ joint.setState(State.ACTIVE);
+ if (joint.getType().equals(IFeedJoint.FeedJointType.COMPUTE)) {
+ cInfo.setComputeFeedJoint(joint);
+ }
+ }
+ }
+ cInfo.setState(FeedJobState.ACTIVE);
+
+ // register activity in metadata
+ registerFeedActivity(cInfo);
+ // notify event listeners
+ notifyFeedEventSubscribers(cInfo, FeedLifecycleEvent.FEED_COLLECT_STARTED);
+ }
+
+ private void notifyFeedEventSubscribers(FeedJobInfo jobInfo, FeedLifecycleEvent event) {
+ JobType jobType = jobInfo.getJobType();
+ List<FeedConnectionId> impactedConnections = new ArrayList<FeedConnectionId>();
+ if (jobType.equals(JobType.INTAKE)) {
+ FeedId feedId = ((FeedIntakeInfo) jobInfo).getFeedId();
+ for (FeedConnectionId connId : eventSubscribers.keySet()) {
+ if (connId.getFeedId().equals(feedId)) {
+ impactedConnections.add(connId);
+ }
+ }
+ } else {
+ impactedConnections.add(((FeedConnectJobInfo) jobInfo).getConnectionId());
+ }
+
+ for (FeedConnectionId connId : impactedConnections) {
+ List<IFeedLifecycleEventSubscriber> subscribers = eventSubscribers.get(connId);
+ if (subscribers != null && !subscribers.isEmpty()) {
+ for (IFeedLifecycleEventSubscriber subscriber : subscribers) {
+ subscriber.handleFeedEvent(event);
+ }
+ }
+ }
+ }
+
+ public synchronized void submitFeedConnectionRequest(IFeedJoint feedJoint, final FeedConnectionRequest request)
+ throws Exception {
+ List<String> locations = null;
+ switch (feedJoint.getType()) {
+ case INTAKE:
+ FeedIntakeInfo intakeInfo = intakeJobInfos.get(feedJoint.getOwnerFeedId());
+ locations = intakeInfo.getIntakeLocation();
+ break;
+ case COMPUTE:
+ FeedConnectionId connectionId = feedJoint.getProvider();
+ FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
+ locations = cInfo.getComputeLocations();
+ break;
+ }
+
+ SubscribeFeedWork work = new SubscribeFeedWork(locations.toArray(new String[] {}), request);
+ FeedWorkManager.INSTANCE.submitWork(work, new SubscribeFeedWork.FeedSubscribeWorkEventListener());
+ }
+
+ public IFeedJoint getSourceFeedJoint(FeedConnectionId connectionId) {
+ FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
+ if (cInfo != null) {
+ return cInfo.getSourceFeedJoint();
+ }
+ return null;
+ }
+
+ public Set<FeedConnectionId> getActiveFeedConnections() {
+ Set<FeedConnectionId> activeConnections = new HashSet<FeedConnectionId>();
+ for (FeedConnectJobInfo cInfo : connectJobInfos.values()) {
+ if (cInfo.getState().equals(FeedJobState.ACTIVE)) {
+ activeConnections.add(cInfo.getConnectionId());
+ }
+ }
+ return activeConnections;
+ }
+
+ public boolean isFeedConnectionActive(FeedConnectionId connectionId) {
+ FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
+ if (cInfo != null) {
+ return cInfo.getState().equals(FeedJobState.ACTIVE);
+ }
+ return false;
+ }
+
+ public void setJobState(FeedConnectionId connectionId, FeedJobState jobState) {
+ FeedConnectJobInfo connectJobInfo = connectJobInfos.get(connectionId);
+ connectJobInfo.setState(jobState);
+ }
+
+ public FeedJobState getFeedJobState(FeedConnectionId connectionId) {
+ return connectJobInfos.get(connectionId).getState();
+ }
+
+ private void handleFeedIntakeJobFinishMessage(FeedIntakeInfo intakeInfo, Message message) throws Exception {
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ JobInfo info = hcc.getJobInfo(message.jobId);
+ JobStatus status = info.getStatus();
+ FeedLifecycleEvent event;
+ event = status.equals(JobStatus.FAILURE) ? FeedLifecycleEvent.FEED_INTAKE_FAILURE
+ : FeedLifecycleEvent.FEED_ENDED;
+
+ // remove feed joints
+ deregisterFeedIntakeJob(message.jobId);
+
+ // notify event listeners
+ notifyFeedEventSubscribers(intakeInfo, event);
+
+ }
+
+ private void handleFeedCollectJobFinishMessage(FeedConnectJobInfo cInfo) throws Exception {
+ FeedConnectionId connectionId = cInfo.getConnectionId();
+
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ JobInfo info = hcc.getJobInfo(cInfo.getJobId());
+ JobStatus status = info.getStatus();
+ boolean failure = status != null && status.equals(JobStatus.FAILURE);
+ FeedPolicyAccessor fpa = new FeedPolicyAccessor(cInfo.getFeedPolicy());
+
+ boolean removeJobHistory = !failure;
+ boolean retainSubsription = cInfo.getState().equals(FeedJobState.UNDER_RECOVERY)
+ || (failure && fpa.continueOnHardwareFailure());
+
+ if (!retainSubsription) {
+ IFeedJoint feedJoint = cInfo.getSourceFeedJoint();
+ feedJoint.removeReceiver(connectionId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Subscription " + cInfo.getConnectionId() + " completed successfully. Removed subscription");
+ }
+ removeFeedJointsPostPipelineTermination(cInfo.getConnectionId());
+ }
+
+ if (removeJobHistory) {
+ connectJobInfos.remove(connectionId);
+ jobInfos.remove(cInfo.getJobId());
+ feedIntakeProgressTrackers.remove(cInfo.getConnectionId());
+ }
+ deregisterFeedActivity(cInfo);
+
+ // notify event listeners
+ FeedLifecycleEvent event = failure ? FeedLifecycleEvent.FEED_COLLECT_FAILURE : FeedLifecycleEvent.FEED_ENDED;
+ notifyFeedEventSubscribers(cInfo, event);
+ }
+
+ private void registerFeedActivity(FeedConnectJobInfo cInfo) {
+ Map<String, String> feedActivityDetails = new HashMap<String, String>();
+
+ if (cInfo.getCollectLocations() != null) {
+ feedActivityDetails.put(FeedActivity.FeedActivityDetails.INTAKE_LOCATIONS,
+ StringUtils.join(cInfo.getCollectLocations().iterator(), ','));
+ }
+
+ if (cInfo.getComputeLocations() != null) {
+ feedActivityDetails.put(FeedActivity.FeedActivityDetails.COMPUTE_LOCATIONS,
+ StringUtils.join(cInfo.getComputeLocations().iterator(), ','));
+ }
+
+ if (cInfo.getStorageLocations() != null) {
+ feedActivityDetails.put(FeedActivity.FeedActivityDetails.STORAGE_LOCATIONS,
+ StringUtils.join(cInfo.getStorageLocations().iterator(), ','));
+ }
+
+ String policyName = cInfo.getFeedPolicy().get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
+ feedActivityDetails.put(FeedActivity.FeedActivityDetails.FEED_POLICY_NAME, policyName);
+
+ feedActivityDetails.put(FeedActivity.FeedActivityDetails.FEED_CONNECT_TIMESTAMP, (new Date()).toString());
+ try {
+ FeedActivity feedActivity = new FeedActivity(cInfo.getConnectionId().getFeedId().getDataverse(), cInfo
+ .getConnectionId().getFeedId().getFeedName(), cInfo.getConnectionId().getDatasetName(),
+ feedActivityDetails);
+ CentralFeedManager.getInstance().getFeedLoadManager()
+ .reportFeedActivity(cInfo.getConnectionId(), feedActivity);
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to register feed activity for " + cInfo + " " + e.getMessage());
+ }
+
+ }
+
+ }
+
+ public void deregisterFeedActivity(FeedConnectJobInfo cInfo) {
+ try {
+ CentralFeedManager.getInstance().getFeedLoadManager().removeFeedActivity(cInfo.getConnectionId());
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to deregister feed activity for " + cInfo + " " + e.getMessage());
+ }
+ }
+ }
+
+ public void removeFeedJointsPostPipelineTermination(FeedConnectionId connectionId) {
+ FeedConnectJobInfo cInfo = connectJobInfos.get(connectionId);
+ List<IFeedJoint> feedJoints = feedPipeline.get(connectionId.getFeedId());
+
+ IFeedJoint sourceJoint = cInfo.getSourceFeedJoint();
+ List<FeedConnectionId> all = sourceJoint.getReceivers();
+ boolean removeSourceJoint = all.size() < 2;
+ if (removeSourceJoint) {
+ feedJoints.remove(sourceJoint);
+ }
+
+ IFeedJoint computeJoint = cInfo.getComputeFeedJoint();
+ if (computeJoint != null && computeJoint.getReceivers().size() < 2) {
+ feedJoints.remove(computeJoint);
+ }
+ }
+
+ public boolean isRegisteredFeedJob(JobId jobId) {
+ return jobInfos.get(jobId) != null;
+ }
+
+ public List<String> getFeedComputeLocations(FeedId feedId) {
+ List<IFeedJoint> feedJoints = feedPipeline.get(feedId);
+ for (IFeedJoint joint : feedJoints) {
+ if (joint.getFeedJointKey().getFeedId().equals(feedId)) {
+ return connectJobInfos.get(joint.getProvider()).getComputeLocations();
+ }
+ }
+ return null;
+ }
+
+ public List<String> getFeedStorageLocations(FeedConnectionId connectionId) {
+ return connectJobInfos.get(connectionId).getStorageLocations();
+ }
+
+ public List<String> getFeedCollectLocations(FeedConnectionId connectionId) {
+ return connectJobInfos.get(connectionId).getCollectLocations();
+ }
+
+ public List<String> getFeedIntakeLocations(FeedId feedId) {
+ return intakeJobInfos.get(feedId).getIntakeLocation();
+ }
+
+ public JobId getFeedCollectJobId(FeedConnectionId connectionId) {
+ return connectJobInfos.get(connectionId).getJobId();
+ }
+
+ public void registerFeedEventSubscriber(FeedConnectionId connectionId, IFeedLifecycleEventSubscriber subscriber) {
+ List<IFeedLifecycleEventSubscriber> subscribers = eventSubscribers.get(connectionId);
+ if (subscribers == null) {
+ subscribers = new ArrayList<IFeedLifecycleEventSubscriber>();
+ eventSubscribers.put(connectionId, subscribers);
+ }
+ subscribers.add(subscriber);
+ }
+
+ public void deregisterFeedEventSubscriber(FeedConnectionId connectionId, IFeedLifecycleEventSubscriber subscriber) {
+ List<IFeedLifecycleEventSubscriber> subscribers = eventSubscribers.get(connectionId);
+ if (subscribers != null) {
+ subscribers.remove(subscriber);
+ }
+ }
+
+ //============================
+
+ public boolean isFeedPointAvailable(FeedJointKey feedJointKey) {
+ List<IFeedJoint> joints = feedPipeline.get(feedJointKey.getFeedId());
+ if (joints != null && !joints.isEmpty()) {
+ for (IFeedJoint joint : joints) {
+ if (joint.getFeedJointKey().equals(feedJointKey)) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ public Collection<IFeedJoint> getFeedIntakeJoints() {
+ List<IFeedJoint> intakeFeedPoints = new ArrayList<IFeedJoint>();
+ for (FeedIntakeInfo info : intakeJobInfos.values()) {
+ intakeFeedPoints.add(info.getIntakeFeedJoint());
+ }
+ return intakeFeedPoints;
+ }
+
+ public IFeedJoint getFeedJoint(FeedJointKey feedPointKey) {
+ List<IFeedJoint> joints = feedPipeline.get(feedPointKey.getFeedId());
+ if (joints != null && !joints.isEmpty()) {
+ for (IFeedJoint joint : joints) {
+ if (joint.getFeedJointKey().equals(feedPointKey)) {
+ return joint;
+ }
+ }
+ }
+ return null;
+ }
+
+ public IFeedJoint getAvailableFeedJoint(FeedJointKey feedJointKey) {
+ IFeedJoint feedJoint = getFeedJoint(feedJointKey);
+ if (feedJoint != null) {
+ return feedJoint;
+ } else {
+ String jointKeyString = feedJointKey.getStringRep();
+ List<IFeedJoint> jointsOnPipeline = feedPipeline.get(feedJointKey.getFeedId());
+ IFeedJoint candidateJoint = null;
+ if (jointsOnPipeline != null) {
+ for (IFeedJoint joint : jointsOnPipeline) {
+ if (jointKeyString.contains(joint.getFeedJointKey().getStringRep())) {
+ if (candidateJoint == null) {
+ candidateJoint = joint;
+ } else if (joint.getFeedJointKey().getStringRep()
+ .contains(candidateJoint.getFeedJointKey().getStringRep())) { // found feed point is a super set of the earlier find
+ candidateJoint = joint;
+ }
+ }
+ }
+ }
+ return candidateJoint;
+ }
+ }
+
+ public JobSpecification getCollectJobSpecification(FeedConnectionId connectionId) {
+ return connectJobInfos.get(connectionId).getSpec();
+ }
+
+ public IFeedJoint getFeedPoint(FeedId sourceFeedId, IFeedJoint.FeedJointType type) {
+ List<IFeedJoint> joints = feedPipeline.get(sourceFeedId);
+ for (IFeedJoint joint : joints) {
+ if (joint.getType().equals(type)) {
+ return joint;
+ }
+ }
+ return null;
+ }
+
+ public FeedConnectJobInfo getFeedConnectJobInfo(FeedConnectionId connectionId) {
+ return connectJobInfos.get(connectionId);
+ }
+
+ private void setLocations(FeedConnectJobInfo cInfo) {
+ JobSpecification jobSpec = cInfo.getSpec();
+
+ List<OperatorDescriptorId> collectOperatorIds = 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();
+ boolean computeOp = false;
+ for (IPushRuntimeFactory rf : runtimeFactories) {
+ if (rf instanceof AssignRuntimeFactory) {
+ IConnectorDescriptor connDesc = jobSpec.getOperatorInputMap().get(op.getOperatorId()).get(0);
+ IOperatorDescriptor sourceOp = jobSpec.getConnectorOperatorMap().get(connDesc.getConnectorId())
+ .getLeft().getLeft();
+ if (sourceOp instanceof FeedCollectOperatorDescriptor) {
+ computeOp = true;
+ break;
+ }
+ }
+ }
+ if (computeOp) {
+ computeOperatorIds.add(entry.getKey());
+ }
+ } else if (actualOp instanceof LSMTreeIndexInsertUpdateDeleteOperatorDescriptor) {
+ storageOperatorIds.add(entry.getKey());
+ } else if (actualOp instanceof FeedCollectOperatorDescriptor) {
+ collectOperatorIds.add(entry.getKey());
+ }
+ }
+
+ try {
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ JobInfo info = hcc.getJobInfo(cInfo.getJobId());
+ List<String> collectLocations = new ArrayList<String>();
+ for (OperatorDescriptorId collectOpId : collectOperatorIds) {
+ Map<Integer, String> operatorLocations = info.getOperatorLocations().get(collectOpId);
+ int nOperatorInstances = operatorLocations.size();
+ for (int i = 0; i < nOperatorInstances; i++) {
+ collectLocations.add(operatorLocations.get(i));
+ }
+ }
+
+ List<String> computeLocations = new ArrayList<String>();
+ for (OperatorDescriptorId computeOpId : computeOperatorIds) {
+ Map<Integer, String> operatorLocations = info.getOperatorLocations().get(computeOpId);
+ if (operatorLocations != null) {
+ int nOperatorInstances = operatorLocations.size();
+ for (int i = 0; i < nOperatorInstances; i++) {
+ computeLocations.add(operatorLocations.get(i));
+ }
+ } else {
+ computeLocations.clear();
+ computeLocations.addAll(collectLocations);
+ }
+ }
+
+ List<String> storageLocations = new ArrayList<String>();
+ for (OperatorDescriptorId storageOpId : storageOperatorIds) {
+ Map<Integer, String> operatorLocations = info.getOperatorLocations().get(storageOpId);
+ if (operatorLocations == null) {
+ continue;
+ }
+ int nOperatorInstances = operatorLocations.size();
+ for (int i = 0; i < nOperatorInstances; i++) {
+ storageLocations.add(operatorLocations.get(i));
+ }
+ }
+ cInfo.setCollectLocations(collectLocations);
+ cInfo.setComputeLocations(computeLocations);
+ cInfo.setStorageLocations(storageLocations);
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+
+ }
+}
\ No newline at end of file
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedJoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedJoint.java
new file mode 100644
index 0000000..6ba1b30
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedJoint.java
@@ -0,0 +1,186 @@
+/*
+ * 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.feeds;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionRequest;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedJointKey;
+import edu.uci.ics.asterix.common.feeds.api.IFeedJoint;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleListener.ConnectionLocation;
+
+public class FeedJoint implements IFeedJoint {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedJoint.class.getName());
+
+ /** A unique key associated with the feed point **/
+ private final FeedJointKey key;
+
+ /** The state associated with the FeedJoint **/
+ private State state;
+
+ /** A list of subscribers that receive data from this FeedJoint **/
+ private final List<FeedConnectionId> receivers;
+
+ /** The feedId on which the feedPoint resides **/
+ private final FeedId ownerFeedId;
+
+ /** A list of feed subscription requests submitted for subscribing to the FeedPoint's data **/
+ private final List<FeedConnectionRequest> connectionRequests;
+
+ private final ConnectionLocation connectionLocation;
+
+ private final FeedJointType type;
+
+ private FeedConnectionId provider;
+
+ public FeedJoint(FeedJointKey key, FeedId ownerFeedId, ConnectionLocation subscriptionLocation, FeedJointType type,
+ FeedConnectionId provider) {
+ this.key = key;
+ this.ownerFeedId = ownerFeedId;
+ this.type = type;
+ this.receivers = new ArrayList<FeedConnectionId>();
+ this.state = State.CREATED;
+ this.connectionLocation = subscriptionLocation;
+ this.connectionRequests = new ArrayList<FeedConnectionRequest>();
+ this.provider = provider;
+ }
+
+ @Override
+ public int hashCode() {
+ return key.hashCode();
+ }
+
+ public void addReceiver(FeedConnectionId connectionId) {
+ receivers.add(connectionId);
+ }
+
+ public void removeReceiver(FeedConnectionId connectionId) {
+ receivers.remove(connectionId);
+ }
+
+ public synchronized void addConnectionRequest(FeedConnectionRequest request) {
+ connectionRequests.add(request);
+ if (state.equals(State.ACTIVE)) {
+ handlePendingConnectionRequest();
+ }
+ }
+
+ public synchronized void setState(State state) {
+ if (this.state.equals(state)) {
+ return;
+ }
+ this.state = state;
+ if (this.state.equals(State.ACTIVE)) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Feed joint " + this + " is now " + State.ACTIVE);
+ }
+ handlePendingConnectionRequest();
+ }
+ }
+
+ private void handlePendingConnectionRequest() {
+ for (FeedConnectionRequest connectionRequest : connectionRequests) {
+ FeedConnectionId connectionId = new FeedConnectionId(connectionRequest.getReceivingFeedId(),
+ connectionRequest.getTargetDataset());
+ try {
+ FeedLifecycleListener.INSTANCE.submitFeedConnectionRequest(this, connectionRequest);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Submitted feed connection request " + connectionRequest + " at feed joint " + this);
+ }
+ addReceiver(connectionId);
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unsuccessful attempt at submitting connection request " + connectionRequest
+ + " at feed joint " + this + ". Message " + e.getMessage());
+ }
+ e.printStackTrace();
+ }
+ }
+ connectionRequests.clear();
+ }
+
+ public FeedConnectionId getReceiver(FeedConnectionId connectionId) {
+ for (FeedConnectionId cid : receivers) {
+ if (cid.equals(connectionId)) {
+ return cid;
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public String toString() {
+ return key.toString() + " [" + connectionLocation + "]" + "[" + state + "]";
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == null) {
+ return false;
+ }
+ if (o == this) {
+ return true;
+ }
+ if (!(o instanceof FeedJoint)) {
+ return false;
+ }
+ return ((FeedJoint) o).getFeedJointKey().equals(this.key);
+ }
+
+ public FeedId getOwnerFeedId() {
+ return ownerFeedId;
+ }
+
+ public List<FeedConnectionRequest> getConnectionRequests() {
+ return connectionRequests;
+ }
+
+ public ConnectionLocation getConnectionLocation() {
+ return connectionLocation;
+ }
+
+ public FeedJointType getType() {
+ return type;
+ }
+
+ @Override
+ public FeedConnectionId getProvider() {
+ return provider;
+ }
+
+ public List<FeedConnectionId> getReceivers() {
+ return receivers;
+ }
+
+ public FeedJointKey getKey() {
+ return key;
+ }
+
+ public synchronized State getState() {
+ return state;
+ }
+
+ @Override
+ public FeedJointKey getFeedJointKey() {
+ return key;
+ }
+
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedLifecycleListener.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedLifecycleListener.java
new file mode 100644
index 0000000..999537f
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedLifecycleListener.java
@@ -0,0 +1,486 @@
+/*
+ * 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.feeds;
+
+import java.io.PrintWriter;
+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.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.SessionConfig;
+import edu.uci.ics.asterix.api.common.SessionConfig.OutputFormat;
+import edu.uci.ics.asterix.aql.base.Statement;
+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.api.IClusterManagementWork;
+import edu.uci.ics.asterix.common.api.IClusterManagementWork.ClusterState;
+import edu.uci.ics.asterix.common.api.IClusterManagementWorkResponse;
+import edu.uci.ics.asterix.common.feeds.FeedConnectJobInfo;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionRequest;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedIntakeInfo;
+import edu.uci.ics.asterix.common.feeds.FeedJobInfo;
+import edu.uci.ics.asterix.common.feeds.FeedJobInfo.FeedJobState;
+import edu.uci.ics.asterix.common.feeds.FeedJointKey;
+import edu.uci.ics.asterix.common.feeds.api.IFeedJoint;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleEventSubscriber;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleListener;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
+import edu.uci.ics.asterix.common.feeds.message.StorageReportFeedMessage;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.ClusterManager;
+import edu.uci.ics.asterix.metadata.feeds.FeedCollectOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.feeds.FeedIntakeOperatorDescriptor;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+/**
+ * A listener that subscribes to events associated with cluster membership
+ * (nodes joining/leaving the cluster) and job lifecycle (start/end of a job).
+ * Subscription to such events allows keeping track of feed ingestion jobs and
+ * take any corrective action that may be required when a node involved in a
+ * feed leaves the cluster.
+ */
+public class FeedLifecycleListener implements IFeedLifecycleListener {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedLifecycleListener.class.getName());
+
+ public static FeedLifecycleListener INSTANCE = new FeedLifecycleListener();
+
+ private final LinkedBlockingQueue<Message> jobEventInbox;
+ private final LinkedBlockingQueue<IClusterManagementWorkResponse> responseInbox;
+ private final Map<FeedCollectInfo, List<String>> dependentFeeds = new HashMap<FeedCollectInfo, List<String>>();
+ private final Map<FeedConnectionId, LinkedBlockingQueue<String>> feedReportQueue;
+ private final FeedJobNotificationHandler feedJobNotificationHandler;
+ private final FeedWorkRequestResponseHandler feedWorkRequestResponseHandler;
+ private final ExecutorService executorService;
+
+ private ClusterState state;
+
+ private FeedLifecycleListener() {
+ this.jobEventInbox = new LinkedBlockingQueue<Message>();
+ this.feedJobNotificationHandler = new FeedJobNotificationHandler(jobEventInbox);
+ this.responseInbox = new LinkedBlockingQueue<IClusterManagementWorkResponse>();
+ this.feedWorkRequestResponseHandler = new FeedWorkRequestResponseHandler(responseInbox);
+ this.feedReportQueue = new HashMap<FeedConnectionId, LinkedBlockingQueue<String>>();
+ this.executorService = Executors.newCachedThreadPool();
+ this.executorService.execute(feedJobNotificationHandler);
+ this.executorService.execute(feedWorkRequestResponseHandler);
+ ClusterManager.INSTANCE.registerSubscriber(this);
+ this.state = AsterixClusterProperties.INSTANCE.getState();
+ }
+
+ @Override
+ public void notifyJobStart(JobId jobId) throws HyracksException {
+ if (feedJobNotificationHandler.isRegisteredFeedJob(jobId)) {
+ jobEventInbox.add(new Message(jobId, Message.MessageKind.JOB_START));
+ }
+ }
+
+ @Override
+ public void notifyJobFinish(JobId jobId) throws HyracksException {
+ if (feedJobNotificationHandler.isRegisteredFeedJob(jobId)) {
+ jobEventInbox.add(new Message(jobId, Message.MessageKind.JOB_FINISH));
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("NO NEED TO NOTIFY JOB FINISH!");
+ }
+ }
+ }
+
+ public FeedConnectJobInfo getFeedConnectJobInfo(FeedConnectionId connectionId) {
+ return feedJobNotificationHandler.getFeedConnectJobInfo(connectionId);
+ }
+
+ public void registerFeedIntakeProgressTracker(FeedConnectionId connectionId,
+ IIntakeProgressTracker feedIntakeProgressTracker) {
+ feedJobNotificationHandler.registerFeedIntakeProgressTracker(connectionId, feedIntakeProgressTracker);
+ }
+
+ public void deregisterFeedIntakeProgressTracker(FeedConnectionId connectionId) {
+ feedJobNotificationHandler.deregisterFeedIntakeProgressTracker(connectionId);
+ }
+
+ public void updateTrackingInformation(StorageReportFeedMessage srm) {
+ feedJobNotificationHandler.updateTrackingInformation(srm);
+ }
+
+ /*
+ * Traverse job specification to categorize job as a feed intake job or a feed collection job
+ */
+ @Override
+ public void notifyJobCreation(JobId jobId, IActivityClusterGraphGeneratorFactory acggf) throws HyracksException {
+ JobSpecification spec = acggf.getJobSpecification();
+ FeedConnectionId feedConnectionId = null;
+ Map<String, String> feedPolicy = null;
+ for (IOperatorDescriptor opDesc : spec.getOperatorMap().values()) {
+ if (opDesc instanceof FeedCollectOperatorDescriptor) {
+ feedConnectionId = ((FeedCollectOperatorDescriptor) opDesc).getFeedConnectionId();
+ feedPolicy = ((FeedCollectOperatorDescriptor) opDesc).getFeedPolicyProperties();
+ feedJobNotificationHandler.registerFeedCollectionJob(
+ ((FeedCollectOperatorDescriptor) opDesc).getSourceFeedId(), feedConnectionId, jobId, spec,
+ feedPolicy);
+ break;
+ } else if (opDesc instanceof FeedIntakeOperatorDescriptor) {
+ feedJobNotificationHandler.registerFeedIntakeJob(((FeedIntakeOperatorDescriptor) opDesc).getFeedId(),
+ jobId, spec);
+ break;
+ }
+ }
+ }
+
+ public void setJobState(FeedConnectionId connectionId, FeedJobState jobState) {
+ feedJobNotificationHandler.setJobState(connectionId, jobState);
+ }
+
+ public FeedJobState getFeedJobState(FeedConnectionId connectionId) {
+ return feedJobNotificationHandler.getFeedJobState(connectionId);
+ }
+
+ public 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;
+ }
+ }
+
+ @Override
+ public Set<IClusterManagementWork> notifyNodeFailure(Set<String> deadNodeIds) {
+ Set<IClusterManagementWork> workToBeDone = new HashSet<IClusterManagementWork>();
+
+ Collection<FeedIntakeInfo> intakeInfos = feedJobNotificationHandler.getFeedIntakeInfos();
+ Collection<FeedConnectJobInfo> connectJobInfos = feedJobNotificationHandler.getFeedConnectInfos();
+
+ Map<String, List<FeedJobInfo>> impactedJobs = new HashMap<String, List<FeedJobInfo>>();
+
+ for (String deadNode : deadNodeIds) {
+ for (FeedIntakeInfo intakeInfo : intakeInfos) {
+ if (intakeInfo.getIntakeLocation().contains(deadNode)) {
+ List<FeedJobInfo> infos = impactedJobs.get(deadNode);
+ if (infos == null) {
+ infos = new ArrayList<FeedJobInfo>();
+ impactedJobs.put(deadNode, infos);
+ }
+ infos.add(intakeInfo);
+ intakeInfo.setState(FeedJobState.UNDER_RECOVERY);
+ }
+ }
+
+ for (FeedConnectJobInfo connectInfo : connectJobInfos) {
+ if (connectInfo.getStorageLocations().contains(deadNode)) {
+ continue;
+ }
+ if (connectInfo.getComputeLocations().contains(deadNode)
+ || connectInfo.getCollectLocations().contains(deadNode)) {
+ List<FeedJobInfo> infos = impactedJobs.get(deadNode);
+ if (infos == null) {
+ infos = new ArrayList<FeedJobInfo>();
+ impactedJobs.put(deadNode, infos);
+ }
+ infos.add(connectInfo);
+ connectInfo.setState(FeedJobState.UNDER_RECOVERY);
+ feedJobNotificationHandler.deregisterFeedActivity(connectInfo);
+ }
+ }
+
+ }
+
+ if (impactedJobs.size() > 0) {
+ AddNodeWork addNodeWork = new AddNodeWork(deadNodeIds, deadNodeIds.size(), this);
+ feedWorkRequestResponseHandler.registerFeedWork(addNodeWork.getWorkId(), impactedJobs);
+ workToBeDone.add(addNodeWork);
+ }
+ return workToBeDone;
+
+ }
+
+ public static class FailureReport {
+
+ private final List<Pair<FeedConnectJobInfo, List<String>>> recoverableConnectJobs;
+ private final Map<IFeedJoint, List<String>> recoverableIntakeFeedIds;
+
+ public FailureReport(Map<IFeedJoint, List<String>> recoverableIntakeFeedIds,
+ List<Pair<FeedConnectJobInfo, List<String>>> recoverableSubscribers) {
+ this.recoverableConnectJobs = recoverableSubscribers;
+ this.recoverableIntakeFeedIds = recoverableIntakeFeedIds;
+ }
+
+ public List<Pair<FeedConnectJobInfo, List<String>>> getRecoverableSubscribers() {
+ return recoverableConnectJobs;
+ }
+
+ public Map<IFeedJoint, List<String>> getRecoverableIntakeFeedIds() {
+ return recoverableIntakeFeedIds;
+ }
+
+ }
+
+ @Override
+ public Set<IClusterManagementWork> notifyNodeJoin(String joinedNodeId) {
+ ClusterState newState = AsterixClusterProperties.INSTANCE.getState();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(joinedNodeId + " joined the cluster. " + "Asterix state: " + newState);
+ }
+
+ boolean needToReActivateFeeds = !newState.equals(state) && (newState == ClusterState.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<FeedCollectInfo> feedsThatCanBeRevived = new ArrayList<FeedCollectInfo>();
+ for (Entry<FeedCollectInfo, 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(ClusterState previousState, ClusterState newState) {
+ switch (newState) {
+ case ACTIVE:
+ if (previousState.equals(ClusterState.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;
+ }
+
+ }
+
+ public static class FeedsDeActivator implements Runnable {
+
+ private List<FeedConnectJobInfo> failedConnectjobs;
+
+ public FeedsDeActivator(List<FeedConnectJobInfo> failedConnectjobs) {
+ this.failedConnectjobs = failedConnectjobs;
+ }
+
+ @Override
+ public void run() {
+ for (FeedConnectJobInfo failedConnectJob : failedConnectjobs) {
+ endFeed(failedConnectJob);
+ }
+ }
+
+ private void endFeed(FeedConnectJobInfo cInfo) {
+ MetadataTransactionContext ctx = null;
+ PrintWriter writer = new PrintWriter(System.out, true);
+ SessionConfig pc = new SessionConfig(writer, OutputFormat.ADM);
+
+ try {
+ ctx = MetadataManager.INSTANCE.beginTransaction();
+ FeedId feedId = cInfo.getConnectionId().getFeedId();
+ DisconnectFeedStatement stmt = new DisconnectFeedStatement(new Identifier(feedId.getDataverse()),
+ new Identifier(feedId.getFeedName()), new Identifier(cInfo.getConnectionId().getDatasetName()));
+ List<Statement> statements = new ArrayList<Statement>();
+ DataverseDecl dataverseDecl = new DataverseDecl(new Identifier(feedId.getDataverse()));
+ statements.add(dataverseDecl);
+ statements.add(stmt);
+ AqlTranslator translator = new AqlTranslator(statements, pc);
+ translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null,
+ AqlTranslator.ResultDelivery.SYNC);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("End irrecoverable feed: " + cInfo.getConnectionId());
+ }
+ MetadataManager.INSTANCE.commitTransaction(ctx);
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Exception in ending loser feed: " + cInfo.getConnectionId() + " 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");
+ }
+ }
+
+ }
+
+ }
+ }
+
+ public void submitFeedConnectionRequest(IFeedJoint feedPoint, FeedConnectionRequest subscriptionRequest)
+ throws Exception {
+ feedJobNotificationHandler.submitFeedConnectionRequest(feedPoint, subscriptionRequest);
+ }
+
+ @Override
+ public List<FeedConnectionId> getActiveFeedConnections(FeedId feedId) {
+ List<FeedConnectionId> connections = new ArrayList<FeedConnectionId>();
+ Collection<FeedConnectionId> activeConnections = feedJobNotificationHandler.getActiveFeedConnections();
+ if (feedId != null) {
+ for (FeedConnectionId connectionId : activeConnections) {
+ if (connectionId.getFeedId().equals(feedId)) {
+ connections.add(connectionId);
+ }
+ }
+ } else {
+ connections.addAll(activeConnections);
+ }
+ return connections;
+ }
+
+ @Override
+ public List<String> getComputeLocations(FeedId feedId) {
+ return feedJobNotificationHandler.getFeedComputeLocations(feedId);
+ }
+
+ @Override
+ public List<String> getIntakeLocations(FeedId feedId) {
+ return feedJobNotificationHandler.getFeedIntakeLocations(feedId);
+ }
+
+ @Override
+ public List<String> getStoreLocations(FeedConnectionId feedConnectionId) {
+ return feedJobNotificationHandler.getFeedStorageLocations(feedConnectionId);
+ }
+
+ @Override
+ public List<String> getCollectLocations(FeedConnectionId feedConnectionId) {
+ return feedJobNotificationHandler.getFeedCollectLocations(feedConnectionId);
+ }
+
+ @Override
+ public boolean isFeedConnectionActive(FeedConnectionId connectionId) {
+ return feedJobNotificationHandler.isFeedConnectionActive(connectionId);
+ }
+
+ public void reportPartialDisconnection(FeedConnectionId connectionId) {
+ feedJobNotificationHandler.removeFeedJointsPostPipelineTermination(connectionId);
+ }
+
+ 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);
+ }
+
+ @Override
+ public IFeedJoint getAvailableFeedJoint(FeedJointKey feedJointKey) {
+ return feedJobNotificationHandler.getAvailableFeedJoint(feedJointKey);
+ }
+
+ @Override
+ public boolean isFeedJointAvailable(FeedJointKey feedJointKey) {
+ return feedJobNotificationHandler.isFeedPointAvailable(feedJointKey);
+ }
+
+ public void registerFeedJoint(IFeedJoint feedJoint) {
+ feedJobNotificationHandler.registerFeedJoint(feedJoint);
+ }
+
+ public IFeedJoint getFeedJoint(FeedJointKey feedJointKey) {
+ return feedJobNotificationHandler.getFeedJoint(feedJointKey);
+ }
+
+ public void registerFeedEventSubscriber(FeedConnectionId connectionId, IFeedLifecycleEventSubscriber subscriber) {
+ feedJobNotificationHandler.registerFeedEventSubscriber(connectionId, subscriber);
+ }
+
+ public void deregisterFeedEventSubscriber(FeedConnectionId connectionId, IFeedLifecycleEventSubscriber subscriber) {
+ feedJobNotificationHandler.deregisterFeedEventSubscriber(connectionId, subscriber);
+
+ }
+
+ public JobSpecification getCollectJobSpecification(FeedConnectionId connectionId) {
+ return feedJobNotificationHandler.getCollectJobSpecification(connectionId);
+ }
+
+ public JobId getFeedCollectJobId(FeedConnectionId connectionId) {
+ return feedJobNotificationHandler.getFeedCollectJobId(connectionId);
+ }
+
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedLoadManager.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedLoadManager.java
new file mode 100644
index 0000000..cf94b28
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedLoadManager.java
@@ -0,0 +1,298 @@
+/*
+ * 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.feeds;
+
+import java.util.ArrayList;
+import java.util.Collection;
+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 java.util.TreeSet;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedActivity;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedJobInfo.FeedJobState;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.NodeLoadReport;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLoadManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedTrackingManager;
+import edu.uci.ics.asterix.common.feeds.message.FeedCongestionMessage;
+import edu.uci.ics.asterix.common.feeds.message.FeedReportMessage;
+import edu.uci.ics.asterix.common.feeds.message.ScaleInReportMessage;
+import edu.uci.ics.asterix.common.feeds.message.ThrottlingEnabledFeedMessage;
+import edu.uci.ics.asterix.file.FeedOperations;
+import edu.uci.ics.asterix.metadata.feeds.FeedUtil;
+import edu.uci.ics.asterix.metadata.feeds.PrepareStallMessage;
+import edu.uci.ics.asterix.metadata.feeds.TerminateDataFlowMessage;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class FeedLoadManager implements IFeedLoadManager {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedLoadManager.class.getName());
+
+ private static final long MIN_MODIFICATION_INTERVAL = 180000; // 10 seconds
+ private final TreeSet<NodeLoadReport> nodeReports;
+ private final Map<FeedConnectionId, FeedActivity> feedActivities;
+ private final Map<String, Pair<Integer, Integer>> feedMetrics;
+
+ private FeedConnectionId lastModified;
+ private long lastModifiedTimestamp;
+
+ private static final int UNKNOWN = -1;
+
+ public FeedLoadManager() {
+ this.nodeReports = new TreeSet<NodeLoadReport>();
+ this.feedActivities = new HashMap<FeedConnectionId, FeedActivity>();
+ this.feedMetrics = new HashMap<String, Pair<Integer, Integer>>();
+ }
+
+ @Override
+ public void submitNodeLoadReport(NodeLoadReport report) {
+ nodeReports.remove(report);
+ nodeReports.add(report);
+ }
+
+ @Override
+ public void reportCongestion(FeedCongestionMessage message) throws AsterixException {
+ FeedRuntimeId runtimeId = message.getRuntimeId();
+ FeedJobState jobState = FeedLifecycleListener.INSTANCE.getFeedJobState(message.getConnectionId());
+ if (jobState == null
+ || (jobState.equals(FeedJobState.UNDER_RECOVERY))
+ || (message.getConnectionId().equals(lastModified) && System.currentTimeMillis()
+ - lastModifiedTimestamp < MIN_MODIFICATION_INTERVAL)) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Ignoring congestion report from " + runtimeId);
+ }
+ return;
+ } else {
+ try {
+ FeedLifecycleListener.INSTANCE.setJobState(message.getConnectionId(), FeedJobState.UNDER_RECOVERY);
+ int inflowRate = message.getInflowRate();
+ int outflowRate = message.getOutflowRate();
+ List<String> currentComputeLocations = new ArrayList<String>();
+ currentComputeLocations.addAll(FeedLifecycleListener.INSTANCE.getComputeLocations(message
+ .getConnectionId().getFeedId()));
+ int computeCardinality = currentComputeLocations.size();
+ int requiredCardinality = (int) Math
+ .ceil((double) ((computeCardinality * inflowRate) / (double) outflowRate)) + 5;
+ int additionalComputeNodes = requiredCardinality - computeCardinality;
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("INCREASING COMPUTE CARDINALITY from " + computeCardinality + " by "
+ + additionalComputeNodes);
+ }
+
+ List<String> helperComputeNodes = getNodeForSubstitution(additionalComputeNodes);
+
+ // Step 1) Alter the original feed job to adjust the cardinality
+ JobSpecification jobSpec = FeedLifecycleListener.INSTANCE.getCollectJobSpecification(message
+ .getConnectionId());
+ helperComputeNodes.addAll(currentComputeLocations);
+ List<String> newLocations = new ArrayList<String>();
+ newLocations.addAll(currentComputeLocations);
+ newLocations.addAll(helperComputeNodes);
+ FeedUtil.increaseCardinality(jobSpec, FeedRuntimeType.COMPUTE, requiredCardinality, newLocations);
+
+ // Step 2) send prepare to stall message
+ gracefullyTerminateDataFlow(message.getConnectionId(), Integer.MAX_VALUE);
+
+ // Step 3) run the altered job specification
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("New Job after adjusting to the workload " + jobSpec);
+ }
+
+ Thread.sleep(10000);
+ runJob(jobSpec, false);
+ lastModified = message.getConnectionId();
+ lastModifiedTimestamp = System.currentTimeMillis();
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Unable to form the required job for scaling in/out" + e.getMessage());
+ }
+ throw new AsterixException(e);
+ }
+ }
+ }
+
+ @Override
+ public void submitScaleInPossibleReport(ScaleInReportMessage message) throws Exception {
+ FeedJobState jobState = FeedLifecycleListener.INSTANCE.getFeedJobState(message.getConnectionId());
+ if (jobState == null || (jobState.equals(FeedJobState.UNDER_RECOVERY))) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("JobState information for job " + "[" + message.getConnectionId() + "]" + " not found ");
+ }
+ return;
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Processing scale-in message " + message);
+ }
+ FeedLifecycleListener.INSTANCE.setJobState(message.getConnectionId(), FeedJobState.UNDER_RECOVERY);
+ JobSpecification jobSpec = FeedLifecycleListener.INSTANCE.getCollectJobSpecification(message
+ .getConnectionId());
+ int reducedCardinality = message.getReducedCardinaliy();
+ List<String> currentComputeLocations = new ArrayList<String>();
+ currentComputeLocations.addAll(FeedLifecycleListener.INSTANCE.getComputeLocations(message.getConnectionId()
+ .getFeedId()));
+ FeedUtil.decreaseComputeCardinality(jobSpec, FeedRuntimeType.COMPUTE, reducedCardinality,
+ currentComputeLocations);
+
+ gracefullyTerminateDataFlow(message.getConnectionId(), reducedCardinality - 1);
+ Thread.sleep(3000);
+ JobId newJobId = runJob(jobSpec, false);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Launch modified job" + "[" + newJobId + "]" + "for scale-in \n" + jobSpec);
+ }
+
+ }
+ }
+
+ private void gracefullyTerminateDataFlow(FeedConnectionId connectionId, int computePartitionRetainLimit)
+ throws Exception {
+ // Step 1) send prepare to stall message
+ PrepareStallMessage stallMessage = new PrepareStallMessage(connectionId, computePartitionRetainLimit);
+ List<String> intakeLocations = FeedLifecycleListener.INSTANCE.getCollectLocations(connectionId);
+ List<String> computeLocations = FeedLifecycleListener.INSTANCE.getComputeLocations(connectionId.getFeedId());
+ List<String> storageLocations = FeedLifecycleListener.INSTANCE.getStoreLocations(connectionId);
+
+ Set<String> operatorLocations = new HashSet<String>();
+
+ operatorLocations.addAll(intakeLocations);
+ operatorLocations.addAll(computeLocations);
+ operatorLocations.addAll(storageLocations);
+
+ JobSpecification messageJobSpec = FeedOperations.buildPrepareStallMessageJob(stallMessage, operatorLocations);
+ runJob(messageJobSpec, true);
+
+ // Step 2)
+ TerminateDataFlowMessage terminateMesg = new TerminateDataFlowMessage(connectionId);
+ messageJobSpec = FeedOperations.buildTerminateFlowMessageJob(terminateMesg, intakeLocations);
+ runJob(messageJobSpec, true);
+ }
+
+ public static JobId runJob(JobSpecification spec, boolean waitForCompletion) throws Exception {
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ JobId jobId = hcc.startJob(spec);
+ if (waitForCompletion) {
+ hcc.waitForCompletion(jobId);
+ }
+ return jobId;
+ }
+
+ @Override
+ public void submitFeedRuntimeReport(FeedReportMessage report) {
+ String key = "" + report.getConnectionId() + ":" + report.getRuntimeId().getFeedRuntimeType();
+ Pair<Integer, Integer> value = feedMetrics.get(key);
+ if (value == null) {
+ value = new Pair<Integer, Integer>(report.getValue(), 1);
+ feedMetrics.put(key, value);
+ } else {
+ value.first = value.first + report.getValue();
+ value.second = value.second + 1;
+ }
+ }
+
+ @Override
+ public int getOutflowRate(FeedConnectionId connectionId, FeedRuntimeType runtimeType) {
+ int rVal;
+ String key = "" + connectionId + ":" + runtimeType;
+ feedMetrics.get(key);
+ Pair<Integer, Integer> value = feedMetrics.get(key);
+ if (value == null) {
+ rVal = UNKNOWN;
+ } else {
+ rVal = value.first / value.second;
+ }
+ return rVal;
+ }
+
+ private List<String> getNodeForSubstitution(int nRequired) {
+ List<String> nodeIds = new ArrayList<String>();
+ Iterator<NodeLoadReport> it = null;
+ int nAdded = 0;
+ while (nAdded < nRequired) {
+ it = nodeReports.iterator();
+ while (it.hasNext()) {
+ nodeIds.add(it.next().getNodeId());
+ nAdded++;
+ }
+ }
+ return nodeIds;
+ }
+
+ @Override
+ public synchronized List<String> getNodes(int required) {
+ Iterator<NodeLoadReport> it;
+ List<String> allocated = new ArrayList<String>();
+ while (allocated.size() < required) {
+ it = nodeReports.iterator();
+ while (it.hasNext() && allocated.size() < required) {
+ allocated.add(it.next().getNodeId());
+ }
+ }
+ return allocated;
+ }
+
+ @Override
+ public void reportThrottlingEnabled(ThrottlingEnabledFeedMessage mesg) throws AsterixException, Exception {
+ System.out.println("Throttling Enabled for " + mesg.getConnectionId() + " " + mesg.getFeedRuntimeId());
+ FeedConnectionId connectionId = mesg.getConnectionId();
+ List<String> destinationLocations = new ArrayList<String>();
+ List<String> storageLocations = FeedLifecycleListener.INSTANCE.getStoreLocations(connectionId);
+ List<String> collectLocations = FeedLifecycleListener.INSTANCE.getCollectLocations(connectionId);
+
+ destinationLocations.addAll(storageLocations);
+ destinationLocations.addAll(collectLocations);
+ JobSpecification messageJobSpec = FeedOperations.buildNotifyThrottlingEnabledMessageJob(mesg,
+ destinationLocations);
+ runJob(messageJobSpec, true);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.warning("Acking disabled for " + mesg.getConnectionId() + " in view of activated throttling");
+ }
+ IFeedTrackingManager trackingManager = CentralFeedManager.getInstance().getFeedTrackingManager();
+ trackingManager.disableAcking(connectionId);
+ }
+
+ @Override
+ public void reportFeedActivity(FeedConnectionId connectionId, FeedActivity activity) {
+ feedActivities.put(connectionId, activity);
+ }
+
+ @Override
+ public FeedActivity getFeedActivity(FeedConnectionId connectionId) {
+ return feedActivities.get(connectionId);
+ }
+
+ @Override
+ public Collection<FeedActivity> getFeedActivities() {
+ return feedActivities.values();
+ }
+
+ @Override
+ public void removeFeedActivity(FeedConnectionId connectionId) {
+ feedActivities.remove(connectionId);
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedManager.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedManager.java
new file mode 100644
index 0000000..907ea2a
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedManager.java
@@ -0,0 +1,140 @@
+/*
+ * 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.feeds;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.config.AsterixFeedProperties;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedMemoryManager;
+import edu.uci.ics.asterix.common.feeds.FeedMessageService;
+import edu.uci.ics.asterix.common.feeds.FeedMetricCollector;
+import edu.uci.ics.asterix.common.feeds.NodeLoadReportService;
+import edu.uci.ics.asterix.common.feeds.api.IFeedConnectionManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessageService;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetadataManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector;
+import edu.uci.ics.asterix.common.feeds.api.IFeedSubscriptionManager;
+import edu.uci.ics.asterix.metadata.feeds.FeedConnectionManager;
+import edu.uci.ics.asterix.metadata.feeds.FeedSubscriptionManager;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * An implementation of the IFeedManager interface.
+ * Provider necessary central repository for registering/retrieving
+ * artifacts/services associated with a feed.
+ */
+public class FeedManager implements IFeedManager {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedManager.class.getName());
+
+ private final IFeedSubscriptionManager feedSubscriptionManager;
+
+ private final IFeedConnectionManager feedConnectionManager;
+
+ private final IFeedMemoryManager feedMemoryManager;
+
+ private final IFeedMetricCollector feedMetricCollector;
+
+ private final IFeedMetadataManager feedMetadataManager;
+
+ private final IFeedMessageService feedMessageService;
+
+ private final NodeLoadReportService nodeLoadReportService;
+
+ private final AsterixFeedProperties asterixFeedProperties;
+
+ private final String nodeId;
+
+ private final int frameSize;
+
+ public FeedManager(String nodeId, AsterixFeedProperties feedProperties, int frameSize) throws AsterixException, HyracksDataException {
+ this.nodeId = nodeId;
+ this.feedSubscriptionManager = new FeedSubscriptionManager(nodeId);
+ this.feedConnectionManager = new FeedConnectionManager(nodeId);
+ this.feedMetadataManager = new FeedMetadataManager(nodeId);
+ this.feedMemoryManager = new FeedMemoryManager(nodeId, feedProperties, frameSize);
+ String ccClusterIp = AsterixClusterProperties.INSTANCE.getCluster() != null ? AsterixClusterProperties.INSTANCE
+ .getCluster().getMasterNode().getClusterIp() : "localhost";
+ this.feedMessageService = new FeedMessageService(feedProperties, nodeId, ccClusterIp);
+ this.nodeLoadReportService = new NodeLoadReportService(nodeId, this);
+ try {
+ this.feedMessageService.start();
+ this.nodeLoadReportService.start();
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to start feed services " + e.getMessage());
+ }
+ e.printStackTrace();
+ }
+ this.feedMetricCollector = new FeedMetricCollector(nodeId);
+ this.frameSize = frameSize;
+ this.asterixFeedProperties = feedProperties;
+ }
+
+ @Override
+ public IFeedSubscriptionManager getFeedSubscriptionManager() {
+ return feedSubscriptionManager;
+ }
+
+ @Override
+ public IFeedConnectionManager getFeedConnectionManager() {
+ return feedConnectionManager;
+ }
+
+ @Override
+ public IFeedMemoryManager getFeedMemoryManager() {
+ return feedMemoryManager;
+ }
+
+ @Override
+ public IFeedMetricCollector getFeedMetricCollector() {
+ return feedMetricCollector;
+ }
+
+ public int getFrameSize() {
+ return frameSize;
+ }
+
+ @Override
+ public IFeedMetadataManager getFeedMetadataManager() {
+ return feedMetadataManager;
+ }
+
+ @Override
+ public IFeedMessageService getFeedMessageService() {
+ return feedMessageService;
+ }
+
+ @Override
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ @Override
+ public String toString() {
+ return "FeedManager " + "[" + nodeId + "]";
+ }
+
+ @Override
+ public AsterixFeedProperties getAsterixFeedProperties() {
+ return asterixFeedProperties;
+ }
+
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedMessageReceiver.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedMessageReceiver.java
new file mode 100644
index 0000000..e16633f
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedMessageReceiver.java
@@ -0,0 +1,74 @@
+package edu.uci.ics.asterix.feeds;
+
+import java.util.logging.Level;
+
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.FeedTupleCommitAckMessage;
+import edu.uci.ics.asterix.common.feeds.MessageReceiver;
+import edu.uci.ics.asterix.common.feeds.NodeLoadReport;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLoadManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessage.MessageType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedTrackingManager;
+import edu.uci.ics.asterix.common.feeds.message.FeedCongestionMessage;
+import edu.uci.ics.asterix.common.feeds.message.FeedReportMessage;
+import edu.uci.ics.asterix.common.feeds.message.ScaleInReportMessage;
+import edu.uci.ics.asterix.common.feeds.message.StorageReportFeedMessage;
+import edu.uci.ics.asterix.common.feeds.message.ThrottlingEnabledFeedMessage;
+import edu.uci.ics.asterix.feeds.CentralFeedManager.AQLExecutor;
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedBootstrap;
+
+public class FeedMessageReceiver extends MessageReceiver<String> {
+
+ private static boolean initialized;
+
+ private final IFeedLoadManager feedLoadManager;
+ private final IFeedTrackingManager feedTrackingManager;
+
+ public FeedMessageReceiver(CentralFeedManager centralFeedManager) {
+ this.feedLoadManager = centralFeedManager.getFeedLoadManager();
+ this.feedTrackingManager = centralFeedManager.getFeedTrackingManager();
+ }
+
+ @Override
+ public void processMessage(String message) throws Exception {
+ JSONObject obj = new JSONObject(message);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Received message " + obj);
+ }
+ MessageType messageType = MessageType.valueOf(obj.getString(FeedConstants.MessageConstants.MESSAGE_TYPE));
+ switch (messageType) {
+ case XAQL:
+ if (!initialized) {
+ FeedBootstrap.setUpInitialArtifacts();
+ initialized = true;
+ }
+ AQLExecutor.executeAQL(obj.getString(FeedConstants.MessageConstants.AQL));
+ break;
+ case CONGESTION:
+ feedLoadManager.reportCongestion(FeedCongestionMessage.read(obj));
+ break;
+ case FEED_REPORT:
+ feedLoadManager.submitFeedRuntimeReport(FeedReportMessage.read(obj));
+ break;
+ case NODE_REPORT:
+ feedLoadManager.submitNodeLoadReport(NodeLoadReport.read(obj));
+ break;
+ case SCALE_IN_REQUEST:
+ feedLoadManager.submitScaleInPossibleReport(ScaleInReportMessage.read(obj));
+ break;
+ case STORAGE_REPORT:
+ FeedLifecycleListener.INSTANCE.updateTrackingInformation(StorageReportFeedMessage.read(obj));
+ break;
+ case COMMIT_ACK:
+ feedTrackingManager.submitAckReport(FeedTupleCommitAckMessage.read(obj));
+ break;
+ case THROTTLING_ENABLED:
+ feedLoadManager.reportThrottlingEnabled(ThrottlingEnabledFeedMessage.read(obj));
+ default:
+ break;
+ }
+
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedMetadataManager.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedMetadataManager.java
new file mode 100644
index 0000000..bd0c09b
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedMetadataManager.java
@@ -0,0 +1,93 @@
+package edu.uci.ics.asterix.feeds;
+
+import java.util.Date;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetadataManager;
+import edu.uci.ics.asterix.hyracks.bootstrap.FeedBootstrap;
+import edu.uci.ics.asterix.metadata.feeds.XAQLFeedMessage;
+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.types.ARecordType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class FeedMetadataManager implements IFeedMetadataManager {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedMetadataManager.class.getName());
+
+ private final String nodeId;
+ private ARecordType recordType;
+
+ public FeedMetadataManager(String nodeId) throws AsterixException, HyracksDataException {
+ this.nodeId = nodeId;
+ String[] fieldNames = new String[] { "id", "dataverseName", "feedName", "targetDataset", "tuple", "message",
+ "timestamp" };
+ IAType[] fieldTypes = new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
+ BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING };
+
+ recordType = new ARecordType(FeedBootstrap.FAILED_TUPLE_DATASET_TYPE, fieldNames, fieldTypes, true);
+ }
+
+ @Override
+ public void logTuple(FeedConnectionId connectionId, String tuple, String message, IFeedManager feedManager)
+ throws AsterixException {
+ try {
+ AString id = new AString("1");
+ AString dataverseValue = new AString(connectionId.getFeedId().getDataverse());
+ AString feedValue = new AString(connectionId.getFeedId().getFeedName());
+ AString targetDatasetValue = new AString(connectionId.getDatasetName());
+ AString tupleValue = new AString(tuple);
+ AString messageValue = new AString(message);
+ AString dateTime = new AString(new Date().toString());
+
+ IAObject[] fields = new IAObject[] { id, dataverseValue, feedValue, targetDatasetValue, tupleValue,
+ messageValue, dateTime };
+ ARecord record = new ARecord(recordType, fields);
+ StringBuilder builder = new StringBuilder();
+ builder.append("use dataverse " + FeedBootstrap.FEEDS_METADATA_DV + ";" + "\n");
+ builder.append("insert into dataset " + FeedBootstrap.FAILED_TUPLE_DATASET + " ");
+ builder.append(" (" + recordToString(record) + ")");
+ builder.append(";");
+
+ XAQLFeedMessage xAqlMessage = new XAQLFeedMessage(connectionId, builder.toString());
+ feedManager.getFeedMessageService().sendMessage(xAqlMessage);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(" Sent " + xAqlMessage.toJSON());
+ }
+ } catch (Exception pe) {
+ throw new AsterixException(pe);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "FeedMetadataManager [" + nodeId + "]";
+ }
+
+ private String recordToString(ARecord record) {
+ String[] fieldNames = record.getType().getFieldNames();
+ StringBuilder sb = new StringBuilder();
+ sb.append("{ ");
+ for (int i = 0; i < fieldNames.length; i++) {
+ if (i > 0) {
+ sb.append(", ");
+ }
+ sb.append("\"" + fieldNames[i] + "\"");
+ sb.append(": ");
+ switch (record.getType().getFieldTypes()[i].getTypeTag()) {
+ case STRING:
+ sb.append("\"" + ((AString) record.getValueByPos(i)).getStringValue() + "\"");
+ break;
+ }
+ }
+ sb.append(" }");
+ return sb.toString();
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedTrackingManager.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedTrackingManager.java
new file mode 100644
index 0000000..c55c132
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedTrackingManager.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.feeds;
+
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedTupleCommitAckMessage;
+import edu.uci.ics.asterix.common.feeds.FeedTupleCommitResponseMessage;
+import edu.uci.ics.asterix.common.feeds.api.IFeedTrackingManager;
+import edu.uci.ics.asterix.file.FeedOperations;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class FeedTrackingManager implements IFeedTrackingManager {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedTrackingManager.class.getName());
+
+ private final BitSet allOnes;
+
+ private Map<FeedConnectionId, Map<AckId, BitSet>> ackHistory;
+ private Map<FeedConnectionId, Map<AckId, Integer>> maxBaseAcked;
+
+ public FeedTrackingManager() {
+ byte[] allOneBytes = new byte[128];
+ Arrays.fill(allOneBytes, (byte) 0xff);
+ allOnes = BitSet.valueOf(allOneBytes);
+ ackHistory = new HashMap<FeedConnectionId, Map<AckId, BitSet>>();
+ maxBaseAcked = new HashMap<FeedConnectionId, Map<AckId, Integer>>();
+ }
+
+ @Override
+ public synchronized void submitAckReport(FeedTupleCommitAckMessage ackMessage) {
+ AckId ackId = getAckId(ackMessage);
+ Map<AckId, BitSet> acksForConnection = ackHistory.get(ackMessage.getConnectionId());
+ if (acksForConnection == null) {
+ acksForConnection = new HashMap<AckId, BitSet>();
+ acksForConnection.put(ackId, BitSet.valueOf(ackMessage.getCommitAcks()));
+ ackHistory.put(ackMessage.getConnectionId(), acksForConnection);
+ }
+ BitSet currentAcks = acksForConnection.get(ackId);
+ if (currentAcks == null) {
+ currentAcks = BitSet.valueOf(ackMessage.getCommitAcks());
+ acksForConnection.put(ackId, currentAcks);
+ } else {
+ currentAcks.or(BitSet.valueOf(ackMessage.getCommitAcks()));
+ }
+ if (Arrays.equals(currentAcks.toByteArray(), allOnes.toByteArray())) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(ackMessage.getIntakePartition() + " (" + ackMessage.getBase() + ")" + " is convered");
+ }
+ Map<AckId, Integer> maxBaseAckedForConnection = maxBaseAcked.get(ackMessage.getConnectionId());
+ if (maxBaseAckedForConnection == null) {
+ maxBaseAckedForConnection = new HashMap<AckId, Integer>();
+ maxBaseAcked.put(ackMessage.getConnectionId(), maxBaseAckedForConnection);
+ }
+ Integer maxBaseAckedValue = maxBaseAckedForConnection.get(ackId);
+ if (maxBaseAckedValue == null) {
+ maxBaseAckedValue = ackMessage.getBase();
+ maxBaseAckedForConnection.put(ackId, ackMessage.getBase());
+ sendCommitResponseMessage(ackMessage.getConnectionId(), ackMessage.getIntakePartition(),
+ ackMessage.getBase());
+ } else if (ackMessage.getBase() == maxBaseAckedValue + 1) {
+ maxBaseAckedForConnection.put(ackId, ackMessage.getBase());
+ sendCommitResponseMessage(ackMessage.getConnectionId(), ackMessage.getIntakePartition(),
+ ackMessage.getBase());
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Ignoring discountiuous acked base " + ackMessage.getBase() + " for " + ackId);
+ }
+ }
+
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("AckId " + ackId + " pending number of acks " + (128 * 8 - currentAcks.cardinality()));
+ }
+ }
+ }
+
+ public synchronized void disableTracking(FeedConnectionId connectionId) {
+ ackHistory.remove(connectionId);
+ maxBaseAcked.remove(connectionId);
+ }
+
+ private void sendCommitResponseMessage(FeedConnectionId connectionId, int partition, int base) {
+ FeedTupleCommitResponseMessage response = new FeedTupleCommitResponseMessage(connectionId, partition, base);
+ List<String> storageLocations = FeedLifecycleListener.INSTANCE.getStoreLocations(connectionId);
+ List<String> collectLocations = FeedLifecycleListener.INSTANCE.getCollectLocations(connectionId);
+ String collectLocation = collectLocations.get(partition);
+ Set<String> messageDestinations = new HashSet<String>();
+ messageDestinations.add(collectLocation);
+ messageDestinations.addAll(storageLocations);
+ try {
+ JobSpecification spec = FeedOperations.buildCommitAckResponseJob(response, messageDestinations);
+ CentralFeedManager.runJob(spec, false);
+ } catch (Exception e) {
+ e.printStackTrace();
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to send commit response message " + response + " exception " + e.getMessage());
+ }
+ }
+ }
+
+ private static AckId getAckId(FeedTupleCommitAckMessage ackMessage) {
+ return new AckId(ackMessage.getConnectionId(), ackMessage.getIntakePartition(), ackMessage.getBase());
+ }
+
+ private static class AckId {
+ private FeedConnectionId connectionId;
+ private int intakePartition;
+ private int base;
+
+ public AckId(FeedConnectionId connectionId, int intakePartition, int base) {
+ this.connectionId = connectionId;
+ this.intakePartition = intakePartition;
+ this.base = base;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof AckId)) {
+ return false;
+ }
+ AckId other = (AckId) o;
+ return other.getConnectionId().equals(connectionId) && other.getIntakePartition() == intakePartition
+ && other.getBase() == base;
+ }
+
+ @Override
+ public String toString() {
+ return connectionId + "[" + intakePartition + "]" + "(" + base + ")";
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public int getIntakePartition() {
+ return intakePartition;
+ }
+
+ public int getBase() {
+ return base;
+ }
+
+ }
+
+ @Override
+ public void disableAcking(FeedConnectionId connectionId) {
+ ackHistory.remove(connectionId);
+ maxBaseAcked.remove(connectionId);
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Acking disabled for " + connectionId);
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedWorkRequestResponseHandler.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedWorkRequestResponseHandler.java
new file mode 100644
index 0000000..834784c
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedWorkRequestResponseHandler.java
@@ -0,0 +1,263 @@
+/*
+ * 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.feeds;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+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.api.IClusterManagementWork;
+import edu.uci.ics.asterix.common.api.IClusterManagementWorkResponse;
+import edu.uci.ics.asterix.common.feeds.FeedConnectJobInfo;
+import edu.uci.ics.asterix.common.feeds.FeedIntakeInfo;
+import edu.uci.ics.asterix.common.feeds.FeedJobInfo;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWork;
+import edu.uci.ics.asterix.metadata.cluster.AddNodeWorkResponse;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+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, Map<String, List<FeedJobInfo>>> feedsWaitingForResponse = new HashMap<Integer, Map<String, List<FeedJobInfo>>>();
+
+ 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();
+ Map<String, String> nodeSubstitution = new HashMap<String, String>();
+ switch (submittedWork.getClusterManagementWorkType()) {
+ case ADD_NODE:
+ AddNodeWork addNodeWork = (AddNodeWork) submittedWork;
+ int workId = addNodeWork.getWorkId();
+ Map<String, List<FeedJobInfo>> failureAnalysis = feedsWaitingForResponse.get(workId);
+ AddNodeWorkResponse resp = (AddNodeWorkResponse) response;
+ List<String> nodesAdded = resp.getNodesAdded();
+ List<String> unsubstitutedNodes = new ArrayList<String>();
+ unsubstitutedNodes.addAll(addNodeWork.getDeadNodes());
+ int nodeIndex = 0;
+
+ /** form a mapping between the failed node and its substitute **/
+ if (nodesAdded != null && nodesAdded.size() > 0) {
+ for (String failedNodeId : addNodeWork.getDeadNodes()) {
+ String substitute = nodesAdded.get(nodeIndex);
+ nodeSubstitution.put(failedNodeId, substitute);
+ nodeIndex = (nodeIndex + 1) % nodesAdded.size();
+ unsubstitutedNodes.remove(failedNodeId);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Node " + substitute + " chosen to substiute lost node " + failedNodeId);
+ }
+ }
+ }
+ if (unsubstitutedNodes.size() > 0) {
+ String[] participantNodes = AsterixClusterProperties.INSTANCE.getParticipantNodes().toArray(
+ new String[] {});
+ nodeIndex = 0;
+ for (String unsubstitutedNode : unsubstitutedNodes) {
+ nodeSubstitution.put(unsubstitutedNode, participantNodes[nodeIndex]);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Node " + participantNodes[nodeIndex] + " chosen to substiute lost node "
+ + unsubstitutedNode);
+ }
+ nodeIndex = (nodeIndex + 1) % participantNodes.length;
+ }
+
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Request " + resp.getWork() + " completed using internal nodes");
+ }
+ }
+
+ // alter failed feed intake jobs
+
+ for (Entry<String, List<FeedJobInfo>> entry : failureAnalysis.entrySet()) {
+ String failedNode = entry.getKey();
+ List<FeedJobInfo> impactedJobInfos = entry.getValue();
+ for (FeedJobInfo info : impactedJobInfos) {
+ JobSpecification spec = info.getSpec();
+ replaceNode(spec, failedNode, nodeSubstitution.get(failedNode));
+ info.setSpec(spec);
+ }
+ }
+
+ Set<FeedIntakeInfo> revisedIntakeJobs = new HashSet<FeedIntakeInfo>();
+ Set<FeedConnectJobInfo> revisedConnectJobInfos = new HashSet<FeedConnectJobInfo>();
+
+ for (List<FeedJobInfo> infos : failureAnalysis.values()) {
+ for (FeedJobInfo info : infos) {
+ switch (info.getJobType()) {
+ case INTAKE:
+ revisedIntakeJobs.add((FeedIntakeInfo) info);
+ break;
+ case FEED_CONNECT:
+ revisedConnectJobInfos.add((FeedConnectJobInfo) info);
+ break;
+ }
+ }
+ }
+
+ IHyracksClientConnection hcc = AsterixAppContextInfo.getInstance().getHcc();
+ try {
+ for (FeedIntakeInfo info : revisedIntakeJobs) {
+ hcc.startJob(info.getSpec());
+ }
+ Thread.sleep(2000);
+ for (FeedConnectJobInfo info : revisedConnectJobInfos) {
+ hcc.startJob(info.getSpec());
+ Thread.sleep(2000);
+ }
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to start revised job post failure");
+ }
+ }
+
+ break;
+ case REMOVE_NODE:
+ throw new IllegalStateException("Invalid work submitted");
+ }
+ }
+ }
+
+ 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, Map<String, List<FeedJobInfo>> impactedJobs) {
+ feedsWaitingForResponse.put(workId, impactedJobs);
+ }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedsActivator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedsActivator.java
new file mode 100644
index 0000000..4b5c8ef
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/feeds/FeedsActivator.java
@@ -0,0 +1,95 @@
+package edu.uci.ics.asterix.feeds;
+
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.api.common.SessionConfig;
+import edu.uci.ics.asterix.api.common.SessionConfig.OutputFormat;
+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.Identifier;
+import edu.uci.ics.asterix.aql.translator.AqlTranslator;
+import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public class FeedsActivator implements Runnable {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedJobNotificationHandler.class.getName());
+
+ private List<FeedCollectInfo> 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<FeedCollectInfo> 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 (FeedCollectInfo 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());
+ }
+ }
+ }
+ }
+ }
+
+ public void reviveFeed(String dataverse, String feedName, String dataset, String feedPolicy) {
+ PrintWriter writer = new PrintWriter(System.out, true);
+ SessionConfig pc = new SessionConfig(writer, OutputFormat.ADM);
+ 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, pc);
+ translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null, AqlTranslator.ResultDelivery.SYNC);
+ 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());
+ }
+ }
+ }
+}
\ No newline at end of file
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 5e52f3e..28ee1f2 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,15 +14,34 @@
*/
package edu.uci.ics.asterix.file;
-import java.util.logging.Logger;
+import java.util.Collection;
+import java.util.List;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedConnectJobInfo;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.feeds.FeedTupleCommitResponseMessage;
+import edu.uci.ics.asterix.common.feeds.api.IFeedJoint;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessage;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.message.EndFeedMessage;
+import edu.uci.ics.asterix.common.feeds.message.ThrottlingEnabledFeedMessage;
+import edu.uci.ics.asterix.feeds.FeedLifecycleListener;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.entities.FeedActivity;
+import edu.uci.ics.asterix.metadata.entities.PrimaryFeed;
+import edu.uci.ics.asterix.metadata.feeds.FeedMessageOperatorDescriptor;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.PrepareStallMessage;
+import edu.uci.ics.asterix.metadata.feeds.TerminateDataFlowMessage;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
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;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
@@ -33,44 +52,200 @@
*/
public class FeedOperations {
- private static final Logger LOGGER = Logger.getLogger(IndexOperations.class.getName());
-
/**
- * @param controlFeedStatement
- * The statement representing the action that describes the
- * action that needs to be taken on the feed. E.g. of actions are
- * stop feed or alter feed.
+ * Builds the job spec for ingesting a (primary) feed from its external source via the feed adaptor.
+ *
+ * @param primaryFeed
* @param metadataProvider
- * An instance of the MetadataProvider
- * @return An instance of JobSpec for the job that would send an appropriate
- * control message to the running feed.
- * @throws AsterixException
- * @throws AlgebricksException
+ * @return JobSpecification the Hyracks job specification for receiving data from external source
+ * @throws Exception
*/
- public static JobSpecification buildDisconnectFeedJobSpec(String dataverseName, String feedName,
- String datasetName, AqlMetadataProvider metadataProvider, FeedActivity feedActivity)
- throws AsterixException, AlgebricksException {
+ public static Pair<JobSpecification, IFeedAdapterFactory> buildFeedIntakeJobSpec(PrimaryFeed primaryFeed,
+ AqlMetadataProvider metadataProvider, FeedPolicyAccessor policyAccessor) throws Exception {
JobSpecification spec = JobSpecificationUtils.createJobSpecification();
- IOperatorDescriptor feedMessenger;
- AlgebricksPartitionConstraint messengerPc;
+ spec.setFrameSize(FeedConstants.JobConstants.DEFAULT_FRAME_SIZE);
+ IFeedAdapterFactory adapterFactory = null;
+ IOperatorDescriptor feedIngestor;
+ AlgebricksPartitionConstraint ingesterPc;
try {
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = metadataProvider
- .buildDisconnectFeedMessengerRuntime(spec, dataverseName, feedName, datasetName, feedActivity);
- feedMessenger = p.first;
- messengerPc = p.second;
+ Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IFeedAdapterFactory> t = metadataProvider
+ .buildFeedIntakeRuntime(spec, primaryFeed, policyAccessor);
+ feedIngestor = t.first;
+ ingesterPc = t.second;
+ adapterFactory = t.third;
} catch (AlgebricksException e) {
+ e.printStackTrace();
throw new AsterixException(e);
}
- AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, feedMessenger, messengerPc);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, feedIngestor, ingesterPc);
NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, nullSink, ingesterPc);
+ spec.connect(new OneToOneConnectorDescriptor(spec), feedIngestor, 0, nullSink, 0);
+ spec.addRoot(nullSink);
+ return new Pair<JobSpecification, IFeedAdapterFactory>(spec, adapterFactory);
+ }
+
+ public static JobSpecification buildDiscontinueFeedSourceSpec(AqlMetadataProvider metadataProvider, FeedId feedId)
+ throws AsterixException, AlgebricksException {
+
+ JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+ IOperatorDescriptor feedMessenger = null;
+ AlgebricksPartitionConstraint messengerPc = null;
+
+ List<String> locations = FeedLifecycleListener.INSTANCE.getIntakeLocations(feedId);
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = buildDiscontinueFeedMessengerRuntime(spec, feedId,
+ locations);
+
+ 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);
- return spec;
+ return spec;
+ }
+
+ /**
+ * Builds the job spec for sending message to an active feed to disconnect it from the
+ * its source.
+ */
+ public static Pair<JobSpecification, Boolean> buildDisconnectFeedJobSpec(AqlMetadataProvider metadataProvider,
+ FeedConnectionId connectionId) throws AsterixException, AlgebricksException {
+
+ JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+ IOperatorDescriptor feedMessenger;
+ AlgebricksPartitionConstraint messengerPc;
+ List<String> locations = null;
+ FeedRuntimeType sourceRuntimeType;
+ try {
+ FeedConnectJobInfo cInfo = FeedLifecycleListener.INSTANCE.getFeedConnectJobInfo(connectionId);
+ IFeedJoint sourceFeedJoint = cInfo.getSourceFeedJoint();
+ IFeedJoint computeFeedJoint = cInfo.getComputeFeedJoint();
+
+ boolean terminateIntakeJob = false;
+ boolean completeDisconnect = computeFeedJoint == null || computeFeedJoint.getReceivers().isEmpty();
+ if (completeDisconnect) {
+ sourceRuntimeType = FeedRuntimeType.INTAKE;
+ locations = cInfo.getCollectLocations();
+ terminateIntakeJob = sourceFeedJoint.getReceivers().size() == 1;
+ } else {
+ locations = cInfo.getComputeLocations();
+ sourceRuntimeType = FeedRuntimeType.COMPUTE;
+ }
+
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = buildDisconnectFeedMessengerRuntime(spec,
+ connectionId, locations, sourceRuntimeType, completeDisconnect, sourceFeedJoint.getOwnerFeedId());
+
+ 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);
+ return new Pair<JobSpecification, Boolean>(spec, terminateIntakeJob);
+
+ } catch (AlgebricksException e) {
+ throw new AsterixException(e);
+ }
+
+ }
+
+ public static JobSpecification buildPrepareStallMessageJob(PrepareStallMessage stallMessage,
+ Collection<String> collectLocations) throws AsterixException {
+ JobSpecification messageJobSpec = JobSpecificationUtils.createJobSpecification();
+ try {
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = FeedOperations.buildSendFeedMessageRuntime(
+ messageJobSpec, stallMessage.getConnectionId(), stallMessage, collectLocations);
+ buildSendFeedMessageJobSpec(p.first, p.second, messageJobSpec);
+ } catch (AlgebricksException ae) {
+ throw new AsterixException(ae);
+ }
+ return messageJobSpec;
+ }
+
+ public static JobSpecification buildNotifyThrottlingEnabledMessageJob(
+ ThrottlingEnabledFeedMessage throttlingEnabledMesg, Collection<String> locations) throws AsterixException {
+ JobSpecification messageJobSpec = JobSpecificationUtils.createJobSpecification();
+ try {
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = FeedOperations.buildSendFeedMessageRuntime(
+ messageJobSpec, throttlingEnabledMesg.getConnectionId(), throttlingEnabledMesg, locations);
+ buildSendFeedMessageJobSpec(p.first, p.second, messageJobSpec);
+ } catch (AlgebricksException ae) {
+ throw new AsterixException(ae);
+ }
+ return messageJobSpec;
+ }
+
+ public static JobSpecification buildTerminateFlowMessageJob(TerminateDataFlowMessage terminateMessage,
+ List<String> collectLocations) throws AsterixException {
+ JobSpecification messageJobSpec = JobSpecificationUtils.createJobSpecification();
+ try {
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = FeedOperations.buildSendFeedMessageRuntime(
+ messageJobSpec, terminateMessage.getConnectionId(), terminateMessage, collectLocations);
+ buildSendFeedMessageJobSpec(p.first, p.second, messageJobSpec);
+ } catch (AlgebricksException ae) {
+ throw new AsterixException(ae);
+ }
+ return messageJobSpec;
+ }
+
+ public static JobSpecification buildCommitAckResponseJob(FeedTupleCommitResponseMessage commitResponseMessage,
+ Collection<String> targetLocations) throws AsterixException {
+ JobSpecification messageJobSpec = JobSpecificationUtils.createJobSpecification();
+ try {
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = FeedOperations.buildSendFeedMessageRuntime(
+ messageJobSpec, commitResponseMessage.getConnectionId(), commitResponseMessage, targetLocations);
+ buildSendFeedMessageJobSpec(p.first, p.second, messageJobSpec);
+ } catch (AlgebricksException ae) {
+ throw new AsterixException(ae);
+ }
+ return messageJobSpec;
+ }
+
+ public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDiscontinueFeedMessengerRuntime(
+ JobSpecification jobSpec, FeedId feedId, List<String> locations) throws AlgebricksException {
+ FeedConnectionId feedConnectionId = new FeedConnectionId(feedId, null);
+ IFeedMessage feedMessage = new EndFeedMessage(feedConnectionId, FeedRuntimeType.INTAKE,
+ feedConnectionId.getFeedId(), true, EndFeedMessage.EndMessageType.DISCONTINUE_SOURCE);
+ return buildSendFeedMessageRuntime(jobSpec, feedConnectionId, feedMessage, locations);
+ }
+
+ private static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildSendFeedMessageRuntime(
+ JobSpecification jobSpec, FeedConnectionId feedConenctionId, IFeedMessage feedMessage,
+ Collection<String> locations) throws AlgebricksException {
+ AlgebricksPartitionConstraint partitionConstraint = new AlgebricksAbsolutePartitionConstraint(
+ locations.toArray(new String[] {}));
+ FeedMessageOperatorDescriptor feedMessenger = new FeedMessageOperatorDescriptor(jobSpec, feedConenctionId,
+ feedMessage);
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedMessenger, partitionConstraint);
+ }
+
+ private static JobSpecification buildSendFeedMessageJobSpec(IOperatorDescriptor operatorDescriptor,
+ AlgebricksPartitionConstraint messengerPc, JobSpecification messageJobSpec) {
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(messageJobSpec, operatorDescriptor,
+ messengerPc);
+ NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(messageJobSpec);
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(messageJobSpec, nullSink, messengerPc);
+ messageJobSpec.connect(new OneToOneConnectorDescriptor(messageJobSpec), operatorDescriptor, 0, nullSink, 0);
+ messageJobSpec.addRoot(nullSink);
+ return messageJobSpec;
+ }
+
+ private static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDisconnectFeedMessengerRuntime(
+ JobSpecification jobSpec, FeedConnectionId feedConenctionId, List<String> locations,
+ FeedRuntimeType sourceFeedRuntimeType, boolean completeDisconnection, FeedId sourceFeedId)
+ throws AlgebricksException {
+ IFeedMessage feedMessage = new EndFeedMessage(feedConenctionId, sourceFeedRuntimeType, sourceFeedId,
+ completeDisconnection, EndFeedMessage.EndMessageType.DISCONNECT_FEED);
+ return buildSendFeedMessageRuntime(jobSpec, feedConenctionId, feedMessage, locations);
}
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixGlobalRecoveryManager.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixGlobalRecoveryManager.java
index 0af6a7e..8e633a9 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixGlobalRecoveryManager.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixGlobalRecoveryManager.java
@@ -19,16 +19,17 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import edu.uci.ics.asterix.common.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.common.api.IClusterManagementWork;
+import edu.uci.ics.asterix.common.api.IClusterManagementWorkResponse;
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalDatasetTransactionState;
import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import edu.uci.ics.asterix.feeds.CentralFeedManager;
import edu.uci.ics.asterix.file.ExternalIndexingOperations;
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.IClusterManagementWorkResponse;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.Dataverse;
@@ -36,14 +37,14 @@
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.metadata.entities.Index;
import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
-import edu.uci.ics.asterix.om.util.AsterixClusterProperties.State;
+import edu.uci.ics.asterix.common.api.IClusterManagementWork.ClusterState;
import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
public class AsterixGlobalRecoveryManager implements IClusterEventsSubscriber {
- private static State state;
+ private static ClusterState state;
private static final Logger LOGGER = Logger.getLogger(AsterixGlobalRecoveryManager.class.getName());
private HyracksConnection hcc;
public static AsterixGlobalRecoveryManager INSTANCE;
@@ -63,8 +64,8 @@
@Override
public Set<IClusterManagementWork> notifyNodeJoin(String joinedNodeId) {
// perform global recovery if state changed to active
- final State newState = AsterixClusterProperties.INSTANCE.getState();
- boolean needToRecover = !newState.equals(state) && (newState == State.ACTIVE);
+ final ClusterState newState = AsterixClusterProperties.INSTANCE.getState();
+ boolean needToRecover = !newState.equals(state) && (newState == ClusterState.ACTIVE);
if (needToRecover) {
Thread recoveryThread = new Thread(new Runnable() {
@Override
@@ -79,7 +80,7 @@
List<Dataverse> dataverses = MetadataManager.INSTANCE.getDataverses(mdTxnCtx);
for (Dataverse dataverse : dataverses) {
if (!dataverse.getDataverseName().equals(MetadataConstants.METADATA_DATAVERSE_NAME)) {
- AqlMetadataProvider metadataProvider = new AqlMetadataProvider(dataverse);
+ AqlMetadataProvider metadataProvider = new AqlMetadataProvider(dataverse, CentralFeedManager.getInstance());
List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx,
dataverse.getDataverseName());
for (Dataset dataset : datasets) {
@@ -206,7 +207,7 @@
}
@Override
- public void notifyStateChange(State previousState, State newState) {
+ public void notifyStateChange(ClusterState previousState, ClusterState newState) {
// Do nothing?
}
}
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 9fa9a76..6fcc248 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
@@ -26,8 +26,6 @@
import edu.uci.ics.asterix.api.http.servlet.AQLAPIServlet;
import edu.uci.ics.asterix.api.http.servlet.ConnectorAPIServlet;
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;
@@ -36,6 +34,9 @@
import edu.uci.ics.asterix.common.api.AsterixThreadFactory;
import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import edu.uci.ics.asterix.common.feeds.api.ICentralFeedManager;
+import edu.uci.ics.asterix.feeds.CentralFeedManager;
+import edu.uci.ics.asterix.feeds.FeedLifecycleListener;
import edu.uci.ics.asterix.metadata.MetadataManager;
import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
import edu.uci.ics.asterix.metadata.bootstrap.AsterixStateProxy;
@@ -55,6 +56,7 @@
private Server webServer;
private Server jsonAPIServer;
private Server feedServer;
+ private ICentralFeedManager centralFeedManager;
private static IAsterixStateProxy proxy;
private ICCApplicationContext appCtx;
@@ -89,6 +91,8 @@
setupFeedServer(externalProperties);
feedServer.start();
+ centralFeedManager = CentralFeedManager.getInstance();
+ centralFeedManager.start();
waitUntilServerStart(webServer);
waitUntilServerStart(jsonAPIServer);
@@ -171,9 +175,7 @@
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 950afe4..c8f778d 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
@@ -23,15 +23,15 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import edu.uci.ics.asterix.common.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.common.api.IClusterManagementWork;
+import edu.uci.ics.asterix.common.api.IClusterManagementWorkResponse;
+import edu.uci.ics.asterix.common.api.IClusterManagementWorkResponse.Status;
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;
@@ -141,8 +141,8 @@
for (IClusterManagementWork w : workSet) {
switch (w.getClusterManagementWorkType()) {
case ADD_NODE:
- if (nodesToAdd < ((AddNodeWork) w).getNumberOfNodes()) {
- nodesToAdd = ((AddNodeWork) w).getNumberOfNodes();
+ if (nodesToAdd < ((AddNodeWork) w).getNumberOfNodesRequested()) {
+ nodesToAdd = ((AddNodeWork) w).getNumberOfNodesRequested();
}
nodeAdditionRequests.add((AddNodeWork) w);
break;
@@ -181,7 +181,7 @@
}
for (AddNodeWork w : nodeAdditionRequests) {
- int n = w.getNumberOfNodes();
+ int n = w.getNumberOfNodesRequested();
List<String> nodesToBeAddedForWork = new ArrayList<String>();
for (int i = 0; i < n && i < addedNodes.size(); i++) {
nodesToBeAddedForWork.add(addedNodes.get(i));
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
index 77581c7..e05f406 100644
--- 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
@@ -20,9 +20,9 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import edu.uci.ics.asterix.common.api.IClusterManagementWork;
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.ClusterManager;
import edu.uci.ics.asterix.metadata.cluster.RemoveNodeWork;
@@ -50,8 +50,8 @@
for (IClusterManagementWork w : workSet) {
switch (w.getClusterManagementWorkType()) {
case ADD_NODE:
- if (nodesToAdd < ((AddNodeWork) w).getNumberOfNodes()) {
- nodesToAdd = ((AddNodeWork) w).getNumberOfNodes();
+ if (nodesToAdd < ((AddNodeWork) w).getNumberOfNodesRequested()) {
+ nodesToAdd = ((AddNodeWork) w).getNumberOfNodesRequested();
}
nodeAdditionRequests.add(w);
break;
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
index d2f3345..62dca1f 100755
--- 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
@@ -124,9 +124,8 @@
List<edu.uci.ics.asterix.metadata.entities.DatasourceAdapter> adapters = MetadataManager.INSTANCE
.getDataverseAdapters(mdTxnCtx, dataverse);
for (edu.uci.ics.asterix.metadata.entities.DatasourceAdapter adapter : adapters) {
- if (adapter.getAdapterIdentifier().getAdapterName().startsWith(libraryName + "#")) {
- MetadataManager.INSTANCE.dropAdapter(mdTxnCtx, dataverse, adapter.getAdapterIdentifier()
- .getAdapterName());
+ if (adapter.getAdapterIdentifier().getName().startsWith(libraryName + "#")) {
+ MetadataManager.INSTANCE.dropAdapter(mdTxnCtx, dataverse, adapter.getAdapterIdentifier().getName());
}
}
@@ -145,11 +144,12 @@
private static void installLibraryIfNeeded(String dataverse, final File libraryDir,
Map<String, List<String>> uninstalledLibs) throws Exception {
- String libraryName = libraryDir.getName();
+ String libraryName = libraryDir.getName().trim();
List<String> uninstalledLibsInDv = uninstalledLibs.get(dataverse);
boolean wasUninstalled = uninstalledLibsInDv != null && uninstalledLibsInDv.contains(libraryName);
MetadataTransactionContext mdTxnCtx = null;
+ MetadataManager.INSTANCE.acquireWriteLatch();
try {
mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
edu.uci.ics.asterix.metadata.entities.Library libraryInMetadata = MetadataManager.INSTANCE.getLibrary(
@@ -168,7 +168,7 @@
ExternalLibrary library = getLibrary(new File(libraryDir + File.separator + libraryDescriptors[0]));
if (libraryDescriptors.length == 0) {
- throw new Exception("No library descriptors defined");
+ throw new Exception("No library descriptor defined");
} else if (libraryDescriptors.length > 1) {
throw new Exception("More than 1 library descriptors defined");
}
@@ -186,12 +186,12 @@
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());
+ dataverse, libraryName + "#" + function.getName().trim(), args.size(), args, function
+ .getReturnType().trim(), function.getDefinition().trim(), library.getLanguage()
+ .trim(), function.getFunctionType().trim());
MetadataManager.INSTANCE.addFunction(mdTxnCtx, f);
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Installed function: " + libraryName + "#" + function.getName());
+ LOGGER.info("Installed function: " + libraryName + "#" + function.getName().trim());
}
}
}
@@ -202,8 +202,8 @@
if (library.getLibraryAdapters() != null) {
for (LibraryAdapter adapter : library.getLibraryAdapters().getLibraryAdapter()) {
- String adapterFactoryClass = adapter.getFactoryClass();
- String adapterName = libraryName + "#" + adapter.getName();
+ String adapterFactoryClass = adapter.getFactoryClass().trim();
+ String adapterName = libraryName + "#" + adapter.getName().trim();
AdapterIdentifier aid = new AdapterIdentifier(dataverse, adapterName);
DatasourceAdapter dsa = new DatasourceAdapter(aid, adapterFactoryClass, AdapterType.EXTERNAL);
MetadataManager.INSTANCE.addAdapter(mdTxnCtx, dsa);
@@ -231,6 +231,8 @@
LOGGER.info("Exception in installing library " + libraryName);
}
MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+ } finally {
+ MetadataManager.INSTANCE.releaseWriteLatch();
}
}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedBootstrap.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedBootstrap.java
new file mode 100644
index 0000000..497281b
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedBootstrap.java
@@ -0,0 +1,50 @@
+package edu.uci.ics.asterix.hyracks.bootstrap;
+
+import edu.uci.ics.asterix.feeds.CentralFeedManager;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+
+public class FeedBootstrap {
+
+ public final static String FEEDS_METADATA_DV = "feeds_metadata";
+ public final static String FAILED_TUPLE_DATASET = "failed_tuple";
+ public final static String FAILED_TUPLE_DATASET_TYPE = "FailedTupleType";
+ public final static String FAILED_TUPLE_DATASET_KEY = "id";
+
+ public static void setUpInitialArtifacts() throws Exception {
+
+ StringBuilder builder = new StringBuilder();
+ try {
+ builder.append("create dataverse " + FEEDS_METADATA_DV + ";" + "\n");
+ builder.append("use dataverse " + FEEDS_METADATA_DV + ";" + "\n");
+
+ builder.append("create type " + FAILED_TUPLE_DATASET_TYPE + " as open { ");
+
+ String[] fieldNames = new String[] { "id", "dataverseName", "feedName", "targetDataset", "tuple",
+ "message", "timestamp" };
+ IAType[] fieldTypes = new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
+ BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING };
+
+ for (int i = 0; i < fieldNames.length; i++) {
+ if (i > 0) {
+ builder.append(",");
+ }
+ builder.append(fieldNames[i] + ":");
+ builder.append(fieldTypes[i].getTypeName());
+ }
+ builder.append("}" + ";" + "\n");
+
+ builder.append("create dataset " + FAILED_TUPLE_DATASET + " " + "(" + FAILED_TUPLE_DATASET_TYPE + ")" + " "
+ + "primary key " + FAILED_TUPLE_DATASET_KEY + " on " + MetadataConstants.METADATA_NODEGROUP_NAME
+ + ";");
+
+ CentralFeedManager.AQLExecutor.executeAQL(builder.toString());
+ } catch (Exception e) {
+ e.printStackTrace();
+ System.out.println("Error: " + builder.toString());
+ throw e;
+ }
+ }
+
+}
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
deleted file mode 100644
index 4cdc91c..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedLifecycleListener.java
+++ /dev/null
@@ -1,1188 +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.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 org.apache.commons.lang3.StringUtils;
-
-import edu.uci.ics.asterix.api.common.APIFramework;
-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.common.feeds.FeedConnectionId;
-import edu.uci.ics.asterix.common.feeds.SuperFeedManager;
-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.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.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;
-
-/**
- * A listener that subscribes to events associated with cluster membership (nodes joining/leaving the cluster)
- * and job lifecycle (start/end of a job). Subscription to such events allows keeping track of feed ingestion jobs
- * and take any corrective action that may be required when a node involved in a feed leaves the cluster.
- */
-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) {
- Map<Integer, String> operatorLocations = info.getOperatorLocations().get(ingestOpId);
- int nOperatorInstances = operatorLocations.size();
- for (int i = 0; i < nOperatorInstances; i++) {
- feedInfo.ingestLocations.add(operatorLocations.get(i));
- }
- }
- StringBuilder computeLocs = new StringBuilder();
- for (OperatorDescriptorId computeOpId : computeOperatorIds) {
- Map<Integer, String> operatorLocations = info.getOperatorLocations().get(computeOpId);
- if (operatorLocations != null) {
- int nOperatorInstances = operatorLocations.size();
- for (int i = 0; i < nOperatorInstances; i++) {
- feedInfo.computeLocations.add(operatorLocations.get(i));
- }
- } else {
- feedInfo.computeLocations.addAll(feedInfo.ingestLocations);
- }
- }
-
- StringBuilder storageLocs = new StringBuilder();
- for (OperatorDescriptorId storageOpId : storageOperatorIds) {
- Map<Integer, String> operatorLocations = info.getOperatorLocations().get(storageOpId);
- int nOperatorInstances = operatorLocations.size();
- for (int i = 0; i < nOperatorInstances; i++) {
- feedInfo.storageLocations.add(operatorLocations.get(i));
- }
- }
-
- ingestLocs.append(StringUtils.join(feedInfo.ingestLocations, ","));
- computeLocs.append(StringUtils.join(feedInfo.computeLocations, ","));
- storageLocs.append(StringUtils.join(feedInfo.storageLocations, ","));
-
- feedActivityDetails.put(FeedActivity.FeedActivityDetails.INGEST_LOCATIONS, ingestLocs.toString());
- feedActivityDetails.put(FeedActivity.FeedActivityDetails.COMPUTE_LOCATIONS, computeLocs.toString());
- feedActivityDetails.put(FeedActivity.FeedActivityDetails.STORAGE_LOCATIONS, storageLocs.toString());
- String policyName = feedInfo.feedPolicy.get(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
- feedActivityDetails.put(FeedActivity.FeedActivityDetails.FEED_POLICY_NAME, policyName);
-
- FeedPolicyAccessor policyAccessor = new FeedPolicyAccessor(feedInfo.feedPolicy);
- if (policyAccessor.collectStatistics() || policyAccessor.isElastic()) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Feed " + feedInfo.feedConnectionId + " requires Super Feed Manager");
- }
- configureSuperFeedManager(feedInfo, feedActivityDetails);
- }
-
- 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 configureSuperFeedManager(FeedInfo feedInfo, Map<String, String> feedActivityDetails) {
- // TODO Auto-generated method stub
- 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));
-
- }
-
- 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.getStatus();
- boolean failure = status != null && status.equals(JobStatus.FAILURE);
- FeedActivityType activityType = FeedActivityType.FEED_END;
- Map<String, String> details = new HashMap<String, String>();
- if (failure) {
- activityType = FeedActivityType.FEED_FAILURE;
- }
- 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) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Exception in handling job fninsh message " + message.jobId + "["
- + message.messageKind + "]" + " for job " + message.jobId);
- }
- } 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 conf = new SessionConfig(writer, SessionConfig.OutputFormat.ADM);
- 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, conf);
- translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null,
- AqlTranslator.ResultDelivery.SYNC);
- 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 conf = new SessionConfig(writer, SessionConfig.OutputFormat.ADM);
- 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, conf);
- translator.compileAndExecute(AsterixAppContextInfo.getInstance().getHcc(), null,
- AqlTranslator.ResultDelivery.SYNC);
- 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
deleted file mode 100644
index fa27e7e..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/FeedWorkRequestResponseHandler.java
+++ /dev/null
@@ -1,337 +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.hyracks.bootstrap;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-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>();
- 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/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java b/asterix-app/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
index 4ee37ba..8eb186e 100644
--- a/asterix-app/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
+++ b/asterix-app/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
@@ -19,7 +19,6 @@
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.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
public class ConstantTupleSourceOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
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 d29f881..3e5b734 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
@@ -114,7 +114,7 @@
@Parameters
public static Collection<Object[]> tests() throws Exception {
Collection<Object[]> testArgs = buildTestsInXml(TestCaseContext.ONLY_TESTSUITE_XML_NAME);
- if (testArgs.size() == 0){
+ if (testArgs.size() == 0) {
testArgs = buildTestsInXml(TestCaseContext.DEFAULT_TESTSUITE_XML_NAME);
}
return testArgs;
@@ -138,6 +138,6 @@
@Test
public void test() throws Exception {
- TestsUtils.executeTest(PATH_ACTUAL, tcCtx, null, false);
+ TestsUtils.executeTest(PATH_ACTUAL, tcCtx, null, false);
}
}
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 325803f..152526f 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,15 +1,14 @@
-[ { "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 13, "PendingOp": 0 }
-, { "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 3, "PendingOp": 0 }
-, { "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ] ], "PrimaryKey": [ [ "DataverseName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 1, "PendingOp": 0 }
-, { "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 14, "PendingOp": 0 }
-, { "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "FeedName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "FeedName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 9, "PendingOp": 0 }
-, { "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "NodeName" ] ], "PrimaryKey": [ [ "NodeName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 5, "PendingOp": 0 }
-, { "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "GroupName" ] ], "PrimaryKey": [ [ "GroupName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 13i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 2i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 8i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 3i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ] ], "PrimaryKey": [ [ "DataverseName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 1i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 14i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "FeedName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "FeedName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 10i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 12i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 7i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 4i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 9i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "NodeName" ] ], "PrimaryKey": [ [ "NodeName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 5i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "GroupName" ] ], "PrimaryKey": [ [ "GroupName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 6i32, "PendingOp": 0i32 }
]
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 f1de8f3..92e755b 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,72 +1,73 @@
-[ { "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "FileNumber", "FieldType": "int32" }, { "FieldName": "FileName", "FieldType": "string" }, { "FieldName": "FileSize", "FieldType": "int64" }, { "FieldName": "FileModTime", "FieldType": "datetime" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdapterName", "FieldType": "string" }, { "FieldName": "AdapterConfiguration", "FieldType": "Field_AdapterConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_AdapterConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_SearchKey_in_IndexRecordType_ItemType" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType_ItemType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int64" }, { "FieldName": "WorkingMemorySize", "FieldType": "int64" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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_EnumValues_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_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "LastRefreshTime", "FieldType": "datetime" }, { "FieldName": "TransactionState", "FieldType": "int32" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "Autogenerated", "FieldType": "boolean" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "binary", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "uuid", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+[ { "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "FileNumber", "FieldType": "int32" }, { "FieldName": "FileName", "FieldType": "string" }, { "FieldName": "FileSize", "FieldType": "int64" }, { "FieldName": "FileModTime", "FieldType": "datetime" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "FeedType", "FieldType": "string" }, { "FieldName": "PrimaryTypeDetails", "FieldType": "Field_PrimaryTypeDetails_in_FeedRecordType" }, { "FieldName": "SecondaryTypeDetails", "FieldType": "Field_SecondaryTypeDetails_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_AdapterConfiguration_in_Type_#1_UnionType_Field_PrimaryTypeDetails_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryTypeDetails_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_PrimaryTypeDetails_in_FeedRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_SearchKey_in_IndexRecordType_ItemType" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType_ItemType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_SecondaryTypeDetails_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_SecondaryTypeDetails_in_FeedRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int64" }, { "FieldName": "WorkingMemorySize", "FieldType": "int64" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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_EnumValues_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_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "LastRefreshTime", "FieldType": "datetime" }, { "FieldName": "TransactionState", "FieldType": "int32" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "Autogenerated", "FieldType": "boolean" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_PrimaryTypeDetails_in_FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "AdapterName", "FieldType": "string" }, { "FieldName": "AdapterConfiguration", "FieldType": "Field_AdapterConfiguration_in_Type_#1_UnionType_Field_PrimaryTypeDetails_in_FeedRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_SecondaryTypeDetails_in_FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "SourceFeedName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "binary", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "uuid", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
]
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 cd629f9..e6a86cf 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,18 +1,17 @@
-[ { "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatatypeName" ], [ "DatasetName" ] ], "IsPrimary": false, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ [ "GroupName" ], [ "DataverseName" ], [ "DatasetName" ] ], "IsPrimary": false, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "NestedDatatypeName" ], [ "TopDatatypeName" ] ], "IsPrimary": false, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ], [ "DatasetName" ], [ "ActivityId" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ [ "NodeName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ [ "GroupName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null" ] }
- ]
\ No newline at end of file
+[ { "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatatypeName" ], [ "DatasetName" ] ], "IsPrimary": false, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ [ "GroupName" ], [ "DataverseName" ], [ "DatasetName" ] ], "IsPrimary": false, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "NestedDatatypeName" ], [ "TopDatatypeName" ] ], "IsPrimary": false, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string", "int32" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ [ "NodeName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ [ "GroupName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string" ] }
+ ]
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 325803f..152526f 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,15 +1,14 @@
-[ { "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "DataTypeName": "CompactionPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 13, "PendingOp": 0 }
-, { "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 3, "PendingOp": 0 }
-, { "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ] ], "PrimaryKey": [ [ "DataverseName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 1, "PendingOp": 0 }
-, { "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 14, "PendingOp": 0 }
-, { "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "FeedName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "FeedName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 9, "PendingOp": 0 }
-, { "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "NodeName" ] ], "PrimaryKey": [ [ "NodeName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "DatasetId": 5, "PendingOp": 0 }
-, { "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "GroupName" ] ], "PrimaryKey": [ [ "GroupName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "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", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 13i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 2i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 8i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 3i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ] ], "PrimaryKey": [ [ "DataverseName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 1i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 14i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Feed", "DataTypeName": "FeedRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "FeedName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "FeedName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 10i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DataTypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 12i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 7i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 4i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Library", "DataTypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 9i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "NodeName" ] ], "PrimaryKey": [ [ "NodeName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 5i32, "PendingOp": 0i32 }
+, { "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "GroupName" ] ], "PrimaryKey": [ [ "GroupName" ] ], "GroupName": "MetadataGroup", "Autogenerated": false, "CompactionPolicy": "prefix", "CompactionPolicyProperties": [ { "Name": "max-mergable-component-size", "Value": "1073741824" }, { "Name": "max-tolerance-component-count", "Value": "5" } ] }, "ExternalDetails": null, "Hints": {{ }}, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "DatasetId": 6i32, "PendingOp": 0i32 }
]
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 f1de8f3..92e755b 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,72 +1,73 @@
-[ { "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "FileNumber", "FieldType": "int32" }, { "FieldName": "FileName", "FieldType": "string" }, { "FieldName": "FileSize", "FieldType": "int64" }, { "FieldName": "FileModTime", "FieldType": "datetime" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "AdapterName", "FieldType": "string" }, { "FieldName": "AdapterConfiguration", "FieldType": "Field_AdapterConfiguration_in_FeedRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_AdapterConfiguration_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_SearchKey_in_IndexRecordType_ItemType" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType_ItemType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int64" }, { "FieldName": "WorkingMemorySize", "FieldType": "int64" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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_EnumValues_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_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "LastRefreshTime", "FieldType": "datetime" }, { "FieldName": "TransactionState", "FieldType": "int32" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "Autogenerated", "FieldType": "boolean" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "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": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "binary", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "uuid", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
-, { "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Wed Aug 20 14:03:26 PDT 2014" }
+[ { "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "FileNumber", "FieldType": "int32" }, { "FieldName": "FileName", "FieldType": "string" }, { "FieldName": "FileSize", "FieldType": "int64" }, { "FieldName": "FileModTime", "FieldType": "datetime" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "FeedName", "FieldType": "string" }, { "FieldName": "Function", "FieldType": "Field_Function_in_FeedRecordType" }, { "FieldName": "FeedType", "FieldType": "string" }, { "FieldName": "PrimaryTypeDetails", "FieldType": "Field_PrimaryTypeDetails_in_FeedRecordType" }, { "FieldName": "SecondaryTypeDetails", "FieldType": "Field_SecondaryTypeDetails_in_FeedRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_AdapterConfiguration_in_Type_#1_UnionType_Field_PrimaryTypeDetails_in_FeedRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryTypeDetails_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_PrimaryTypeDetails_in_FeedRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_SearchKey_in_IndexRecordType_ItemType" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType_ItemType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Field_SecondaryTypeDetails_in_FeedRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_SecondaryTypeDetails_in_FeedRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int64" }, { "FieldName": "WorkingMemorySize", "FieldType": "int64" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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_EnumValues_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_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "LastRefreshTime", "FieldType": "datetime" }, { "FieldName": "TransactionState", "FieldType": "int32" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "Autogenerated", "FieldType": "boolean" }, { "FieldName": "CompactionPolicy", "FieldType": "string" }, { "FieldName": "CompactionPolicyProperties", "FieldType": "Field_CompactionPolicyProperties_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_PrimaryTypeDetails_in_FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "AdapterName", "FieldType": "string" }, { "FieldName": "AdapterConfiguration", "FieldType": "Field_AdapterConfiguration_in_Type_#1_UnionType_Field_PrimaryTypeDetails_in_FeedRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "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": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_SecondaryTypeDetails_in_FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "SourceFeedName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "binary", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "uuid", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
+, { "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015" }
]
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 cd629f9..e6a86cf 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,18 +1,17 @@
-[ { "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatatypeName" ], [ "DatasetName" ] ], "IsPrimary": false, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ [ "GroupName" ], [ "DataverseName" ], [ "DatasetName" ] ], "IsPrimary": false, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "NestedDatatypeName" ], [ "TopDatatypeName" ] ], "IsPrimary": false, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "FeedActivity", "IndexName": "FeedActivity", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ], [ "DatasetName" ], [ "ActivityId" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null", "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ [ "NodeName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null" ] }
-, { "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ [ "GroupName" ] ], "IsPrimary": true, "Timestamp": "Sun Jun 08 13:29:06 PDT 2014", "PendingOp": 0, "SearchKeyType": [ "null" ] }
- ]
\ No newline at end of file
+[ { "DataverseName": "Metadata", "DatasetName": "CompactionPolicy", "IndexName": "CompactionPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "CompactionPolicy" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatatypeName" ], [ "DatasetName" ] ], "IsPrimary": false, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ [ "GroupName" ], [ "DataverseName" ], [ "DatasetName" ] ], "IsPrimary": false, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatatypeName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "NestedDatatypeName" ], [ "TopDatatypeName" ] ], "IsPrimary": false, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "FileNumber" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string", "int32" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string", "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ [ "NodeName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string" ] }
+, { "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ [ "GroupName" ] ], "IsPrimary": true, "Timestamp": "Sat Jun 20 16:50:23 PDT 2015", "PendingOp": 0i32, "SearchKeyType": [ "string" ] }
+ ]
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
index 1d4f619..62423de 100644
--- 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
@@ -22,7 +22,7 @@
}
create type TweetMessageType as closed {
- tweetid: int64,
+ tweetid: string,
user: TwitterUserType,
sender-location: point,
send-time: datetime,
@@ -35,4 +35,4 @@
create feed SyntheticTweetFeed
using twitter_firehose
-(("duration"="5"),("tps"="50"),("tput-duration"="5"),("dataverse-dataset"="feeds:SyntheticTweets"),("mode"="controlled"));
+(("duration"="5"),("tps"="50"),("type-name"="TweetMessageType"),("tput-duration"="5"),("dataverse-dataset"="feeds:SyntheticTweets"),("mode"="controlled"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_06/feeds_06.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_06/feeds_06.1.ddl.aql
index 4403f8b..565f06a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_06/feeds_06.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_06/feeds_06.1.ddl.aql
@@ -44,4 +44,4 @@
create feed client_test_feed
using socket_client
-(("sockets"="127.0.0.1:9009"),("addressType"="IP"),("format"="adm"),("file_splits"="data/twitter/tw_messages_100.adm"));
+(("sockets"="127.0.0.1:9009"),("type-name"="TweetMessageType"),("addressType"="IP"),("format"="adm"),("file_splits"="data/twitter/tw_messages_100.adm"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_07/feeds_07.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_07/feeds_07.1.ddl.aql
index d84c49b..07895b7 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_07/feeds_07.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_07/feeds_07.1.ddl.aql
@@ -23,19 +23,21 @@
}
create type TweetMessageType as closed {
- tweetid: int64,
- user: TwitterUserType,
- sender-location: point,
+ id: int64,
+ user: TwitterUserType,
+ latitude: double,
+ longitude: double,
send-time: datetime,
- referred-topics: {{ string }},
- message-text: string
+ message_text: string,
+ created_at: string,
+ country: string
}
create dataset SyntheticTweets(TweetMessageType)
-primary key tweetid;
+primary key id;
create index locationIdx on SyntheticTweets(sender-location) type rtree;
create feed SyntheticTweetFeed
using twitter_firehose
-(("duration"="5"),("tps"="50"),("tput-duration"="5"),("dataverse-dataset"="feeds:SyntheticTweets"),("mode"="controlled"));
+(("duration"="5"),("tps"="50"),("type-name"="TweetMessageType"),("tput-duration"="5"),("dataverse-dataset"="feeds:SyntheticTweets"),("mode"="controlled"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_08/feeds_08.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_08/feeds_08.1.ddl.aql
index 7758f28..baef218 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_08/feeds_08.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_08/feeds_08.1.ddl.aql
@@ -14,28 +14,29 @@
use dataverse feeds;
create type TwitterUserType as closed {
- screen-name: string,
- lang: string,
+ screen_name: string,
+ language: string,
friends_count: int32,
- statuses_count: int32,
+ status_count: int32,
name: string,
followers_count: int32
}
create type TweetMessageType as closed {
- tweetid: int64,
- user: TwitterUserType,
- sender-location: point,
- send-time: datetime,
- referred-topics: {{ string }},
- message-text: string
+ id: int64,
+ user: TwitterUserType,
+ latitude: double,
+ longitude: double,
+ message_text: string,
+ created_at: string,
+ country: string
}
create dataset SyntheticTweets(TweetMessageType)
-primary key tweetid;
+primary key id;
-create index ngram_index on SyntheticTweets(message-text) type ngram(3);
+create index ngram_index on SyntheticTweets(message_text) type ngram(3);
create feed SyntheticTweetFeed
using twitter_firehose
-(("duration"="5"),("tps"="50"),("tput-duration"="5"),("dataverse-dataset"="feeds:SyntheticTweets"),("mode"="controlled"));
+(("duration"="5"),("tps"="50"),("type-name"="TweetMessageType"),("tput-duration"="5"),("dataverse-dataset"="feeds:SyntheticTweets"),("mode"="controlled"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.1.ddl.aql
new file mode 100644
index 0000000..9ea9b47
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.1.ddl.aql
@@ -0,0 +1,42 @@
+/*
+ * Description : Create a feed using the synthetic feed simulator adapter.
+ Create a dataset that has an associated ngram index.
+ 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.
+ * Issue : 711
+ * Expected Res : Success
+ * Date : 8th Feb 2014
+ */
+
+drop dataverse feeds_09 if exists;
+create dataverse feeds_09;
+use dataverse feeds_09;
+
+create type TwitterUserType as closed {
+ screen_name: string,
+ language: string,
+ friends_count: int32,
+ status_count: int32,
+ name: string,
+ followers_count: int32
+}
+
+create type TweetMessageType as closed {
+ id: int64,
+ user: TwitterUserType,
+ latitude: double,
+ longitude: double,
+ message_text: string,
+ created_at: string,
+ country: string
+}
+
+create dataset SyntheticTweets(TweetMessageType)
+primary key id;
+
+create index message_text on SyntheticTweets(message_text) type btree;
+
+create feed SyntheticTweetFeed
+using twitter_firehose
+(("duration"="5"),("tps"="50"),("tput-duration"="5"),("type-name"="TweetMessageType"),("dataverse-dataset"="feeds:SyntheticTweets"),("mode"="controlled"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.2.update.aql
new file mode 100644
index 0000000..951f571
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.2.update.aql
@@ -0,0 +1,16 @@
+/*
+ * Description : Create a feed using the synthetic feed simulator adapter.
+ Create a dataset that has an associated ngram index.
+ 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.
+ * Issue : 711
+ * Expected Res : Success
+ * Date : 2nd April 2014
+ */
+
+use dataverse feeds_09;
+
+set wait-for-completion-feed "true";
+
+connect feed SyntheticTweetFeed to dataset SyntheticTweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.3.query.aql
new file mode 100644
index 0000000..693f016
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_09/feeds_09.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Description : Create a feed using the synthetic feed simulator adapter.
+ Create a dataset that has an associated ngram index.
+ 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.
+ * Issue : 711
+ * Expected Res : Success
+ * Date : 2nd Feb 2014
+ */
+
+use dataverse feeds_09;
+
+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/feeds_10/feeds_10.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_10/feeds_10.1.ddl.aql
new file mode 100644
index 0000000..13252a9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_10/feeds_10.1.ddl.aql
@@ -0,0 +1,30 @@
+/*
+ * Description : Create a dataset with a secondary btree index.
+ Create a feed that uses the file_feed adapter.
+ The file_feed adapter simulates a feed from a file in the HDFS.
+ Connect the feed to the dataset and verify contents of the dataset post completion.
+ * Issue : 711
+ * Expected Res : Success
+ * Date : 6th Feb 2014
+ */
+
+drop dataverse feeds_10 if exists;
+create dataverse feeds_10;
+use dataverse feeds_10;
+
+create type TweetType as closed {
+ id: string,
+ username : string,
+ location : string,
+ text : string,
+ timestamp : string
+}
+
+create dataset Tweets(TweetType)
+primary key id;
+
+create index usernameIdx on Tweets(username) type btree;
+
+create feed TweetFeed
+using file_feed
+(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("type-name"="TweetType"),("tuple-interval"="10"));
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_10/feeds_10.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_10/feeds_10.2.update.aql
new file mode 100644
index 0000000..a99d234
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_10/feeds_10.2.update.aql
@@ -0,0 +1,15 @@
+/*
+ * Description : Create a dataset with a secondary btree index.
+ Create a feed that uses the file_feed adapter.
+ The file_feed adapter simulates a feed from a file in the HDFS.
+ Connect the feed to the dataset and verify contents of the dataset post completion.
+ * Issue : 711
+ * Expected Res : Success
+ * Date : 6th Feb 2014
+ */
+
+use dataverse feeds_10;
+
+set wait-for-completion-feed "true";
+
+connect feed TweetFeed to dataset Tweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_10/feeds_10.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_10/feeds_10.3.query.aql
new file mode 100644
index 0000000..b7fa39a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_10/feeds_10.3.query.aql
@@ -0,0 +1,16 @@
+/*
+ * Description : Create a dataset with a secondary btree index.
+ Create a feed that uses the file_feed adapter.
+ The file_feed adapter simulates a feed from a file in the HDFS.
+ Connect the feed to the dataset and verify contents of the dataset post completion.
+ * Issue : 711
+ * Expected Res : Success
+ * Date : 6th Feb 2014
+ */
+use dataverse feeds_10;
+
+count(
+for $x in dataset('Tweets')
+order by $x.id
+return $x
+)
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.1.ddl.aql
new file mode 100644
index 0000000..d5a1c92
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.1.ddl.aql
@@ -0,0 +1,27 @@
+/*
+ * Description : Create a feed dataset that uses the feed simulator adapter.
+ Begin ingestion using a fully qualified name and verify contents of the dataset post completion.
+ * Expected Res : Success
+ * Date : 24th Dec 2012
+ */
+drop dataverse feeds if exists;
+create dataverse feeds;
+use dataverse feeds;
+
+
+create type TweetType as closed {
+ id: string,
+ username : string,
+ location : string,
+ text : string,
+ timestamp : string
+}
+
+create dataset Tweets(TweetType)
+primary key id;
+
+create feed TweetFeed
+using file_feed
+(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets.adm"),("format"="adm"),("type-name"="TweetType"),("tuple-interval"="10"));
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.2.update.aql
new file mode 100644
index 0000000..fc71769
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description : Create a feed dataset that uses the feed simulator adapter.
+ Begin ingestion using a fully qualified name and verify contents of the dataset post completion.
+ * Expected Res : Success
+ * Date : 24th Dec 2012
+ */
+
+use dataverse feeds;
+
+set wait-for-completion-feed "true";
+
+connect feed feeds.TweetFeed to dataset Tweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.3.query.aql
new file mode 100644
index 0000000..227913d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_11/feeds_11.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description : Create a feed dataset that uses the feed simulator adapter.
+ Begin ingestion using a fully qualified name and verify contents of the dataset post completion.
+ * Expected Res : Success
+ * Date : 24th Dec 2012
+ */
+use dataverse feeds;
+
+for $x in dataset('Tweets')
+order by $x.id
+return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.1.ddl.aql
new file mode 100644
index 0000000..431973b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.1.ddl.aql
@@ -0,0 +1,30 @@
+/*
+ * Description : Create a feed from the contents of a file using a file_feed adaptor.
+ The contents here contain a duplicate record. Since default ingestion policy requires the feed
+ to recover from failures, feed ingestion should be able to surpaas the tuple with duplicate key
+ without an abort.
+ Verify that all but the duplicate record are inserted into the target dataset.
+ * Expected Res : Success
+ * Date : 3rd Apr 2014
+ */
+drop dataverse feeds_12 if exists;
+create dataverse feeds_12;
+use dataverse feeds_12;
+
+
+create type TweetType as closed {
+ id: string,
+ username : string,
+ location : string,
+ text : string,
+ timestamp : string
+}
+
+create dataset Tweets(TweetType)
+primary key id;
+
+create feed TweetFeed
+using file_feed
+(("fs"="localfs"),("path"="nc1://data/twitter/obamatweets_duplicate.adm"),("format"="adm"),("type-name"="TweetType"),("tuple-interval"="10"));
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.2.update.aql
new file mode 100644
index 0000000..8a127ed
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description : Create a feed dataset that uses the feed simulator adapter.
+ Begin ingestion using a fully qualified name and verify contents of the dataset post completion.
+ * Expected Res : Success
+ * Date : 24th Dec 2012
+ */
+
+use dataverse feeds_12;
+
+set wait-for-completion-feed "true";
+
+connect feed TweetFeed to dataset Tweets;
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.3.query.aql
new file mode 100644
index 0000000..387f2ac
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_12/feeds_12.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description : Create a feed dataset that uses the feed simulator adapter.
+ Begin ingestion using a fully qualified name and verify contents of the dataset post completion.
+ * Expected Res : Success
+ * Date : 24th Dec 2012
+ */
+use dataverse feeds_12;
+
+for $x in dataset('Tweets')
+order by $x.id
+return $x
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 896a250..1306499 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,2 +1,2 @@
-[ { "DataverseName": "feeds", "FeedName": "TweetFeed", "AdapterName": "file_feed", "AdapterConfiguration": {{ { "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": "Tue Sep 24 22:30:47 PDT 2013" }
+[ { "DataverseName": "feeds", "FeedName": "TweetFeed", "Function": null, "FeedType": "PRIMARY", "PrimaryTypeDetails": { "AdapterName": "file_feed", "AdapterConfiguration": {{ { "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" } }} }, "SecondaryTypeDetails": null, "Timestamp": "Sat Jun 20 13:55:58 PDT 2015" }
]
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 9dfeb95..177a0f1 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,2 +1,2 @@
-[ { "DataverseName": "feeds", "FeedName": "TweetFeed", "AdapterName": "file_feed", "AdapterConfiguration": {{ { "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": "Tue Sep 24 22:35:03 PDT 2013" }
+[ { "DataverseName": "feeds", "FeedName": "TweetFeed", "Function": "feed_processor", "FeedType": "PRIMARY", "PrimaryTypeDetails": { "AdapterName": "file_feed", "AdapterConfiguration": {{ { "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" } }} }, "SecondaryTypeDetails": null, "Timestamp": "Sat Jun 20 13:55:59 PDT 2015" }
]
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.1.adm b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_09/feeds_09.1.adm
similarity index 100%
rename from asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.1.adm
rename to asterix-app/src/test/resources/runtimets/results/feeds/feeds_09/feeds_09.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_10/feeds_10.1.adm b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_10/feeds_10.1.adm
new file mode 100644
index 0000000..3904598
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_10/feeds_10.1.adm
@@ -0,0 +1,2 @@
+[ 12
+ ]
diff --git a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_11/feeds_11.1.adm b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_11/feeds_11.1.adm
new file mode 100644
index 0000000..00ec0ac
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_11/feeds_11.1.adm
@@ -0,0 +1,13 @@
+[ { "id": "nc1:1", "username": "BronsonMike", "location": "", "text": "@GottaLaff @reutersus Christie and obama just foul weather friends", "timestamp": "Thu Dec 06 16:53:06 PST 2012" }
+, { "id": "nc1:100", "username": "KidrauhlProuds", "location": "", "text": "RT @01Direclieber: A filha do Michael Jackson uma Belieber,a filha do Eminem e uma Belieber,as filhas de Obama sao Beliebers, e a filha do meu pai e Belieber", "timestamp": "Thu Dec 06 16:53:16 PST 2012" }
+, { "id": "nc1:102", "username": "jaysauce82", "location": "", "text": "Not voting for President Obama #BadDecision", "timestamp": "Thu Dec 06 16:53:16 PST 2012" }
+, { "id": "nc1:104", "username": "princeofsupras", "location": "", "text": "RT @01Direclieber: A filha do Michael Jackson e uma Belieber,a filha do Eminem e uma Belieber,as filhas de Obama sao Beliebers, e a filha do meu pai e Belieber", "timestamp": "Thu Dec 06 16:53:15 PST 2012" }
+, { "id": "nc1:106", "username": "GulfDogs", "location": "", "text": "Obama Admin Knew Libyan Terrorists Had US-Provided Weaponsteaparty #tcot #ccot #NewGuards #BreitbartArmy #patriotwttp://t.co/vJxzrQUE", "timestamp": "Thu Dec 06 16:53:14 PST 2012" }
+, { "id": "nc1:108", "username": "Laugzpz", "location": "", "text": "@AlfredoJalife Maestro Obama se hace de la vista gorda, es un acuerdo de siempre creo yo.", "timestamp": "Thu Dec 06 16:53:14 PST 2012" }
+, { "id": "nc1:11", "username": "magarika", "location": "", "text": "RT @ken24xavier: Obama tells SOROS - our plan is ALMOST finished http://t.co/WvzK0GtU", "timestamp": "Thu Dec 06 16:53:05 PST 2012" }
+, { "id": "nc1:111", "username": "ToucanMall", "location": "", "text": "RT @WorldWar3Watch: Michelle Obama Gets More Grammy Nominations Than Justin ... #Obama #WW3 http://t.co/0Wv2GKij", "timestamp": "Thu Dec 06 16:53:13 PST 2012" }
+, { "id": "nc1:113", "username": "ToucanMall", "location": "", "text": "RT @ObamaPalooza: Tiffany Shared What $2,000 Meant to Her ... and the President Stopped by to Talk About It http://t.co/sgT7lsNV #Obama", "timestamp": "Thu Dec 06 16:53:12 PST 2012" }
+, { "id": "nc1:115", "username": "thewildpitch", "location": "", "text": "RT @RevkahJC: Dennis Miller: Obama Should Just Say He Wants To Tax Successful People http://t.co/Ihlemy9Y", "timestamp": "Thu Dec 06 16:53:11 PST 2012" }
+, { "id": "nc1:117", "username": "Rnugent24", "location": "", "text": "RT @ConservativeQuo: unemployment is above 8% again. I wonder how long it will take for Obama to start blaming Bush? 3-2-1 #tcot #antiobama", "timestamp": "Thu Dec 06 16:53:10 PST 2012" }
+, { "id": "nc1:119", "username": "ToucanMall", "location": "", "text": "RT @Newitrsdotcom: I hope #Obama will win re-election... Other four years without meaningless #wars", "timestamp": "Thu Dec 06 16:53:09 PST 2012" }
+ ]
diff --git a/asterix-app/src/test/resources/runtimets/results/feeds/feeds_12/feeds_12.1.adm b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_12/feeds_12.1.adm
new file mode 100644
index 0000000..4905871
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_12/feeds_12.1.adm
@@ -0,0 +1,12 @@
+[ { "id": "nc1:1", "username": "BronsonMike", "location": "", "text": "@GottaLaff @reutersus Christie and obama just foul weather friends", "timestamp": "Thu Dec 06 16:53:06 PST 2012" }
+, { "id": "nc1:100", "username": "KidrauhlProuds", "location": "", "text": "RT @01Direclieber: A filha do Michael Jackson uma Belieber,a filha do Eminem e uma Belieber,as filhas de Obama sao Beliebers, e a filha do meu pai e Belieber", "timestamp": "Thu Dec 06 16:53:16 PST 2012" }
+, { "id": "nc1:102", "username": "jaysauce82", "location": "", "text": "Not voting for President Obama #BadDecision", "timestamp": "Thu Dec 06 16:53:16 PST 2012" }
+, { "id": "nc1:104", "username": "princeofsupras", "location": "", "text": "RT @01Direclieber: A filha do Michael Jackson e uma Belieber,a filha do Eminem e uma Belieber,as filhas de Obama sao Beliebers, e a filha do meu pai e Belieber", "timestamp": "Thu Dec 06 16:53:15 PST 2012" }
+, { "id": "nc1:106", "username": "GulfDogs", "location": "", "text": "Obama Admin Knew Libyan Terrorists Had US-Provided Weaponsteaparty #tcot #ccot #NewGuards #BreitbartArmy #patriotwttp://t.co/vJxzrQUE", "timestamp": "Thu Dec 06 16:53:14 PST 2012" }
+, { "id": "nc1:108", "username": "Laugzpz", "location": "", "text": "@AlfredoJalife Maestro Obama se hace de la vista gorda, es un acuerdo de siempre creo yo.", "timestamp": "Thu Dec 06 16:53:14 PST 2012" }
+, { "id": "nc1:11", "username": "magarika", "location": "", "text": "RT @ken24xavier: Obama tells SOROS - our plan is ALMOST finished http://t.co/WvzK0GtU", "timestamp": "Thu Dec 06 16:53:05 PST 2012" }
+, { "id": "nc1:111", "username": "ToucanMall", "location": "", "text": "RT @WorldWar3Watch: Michelle Obama Gets More Grammy Nominations Than Justin ... #Obama #WW3 http://t.co/0Wv2GKij", "timestamp": "Thu Dec 06 16:53:13 PST 2012" }
+, { "id": "nc1:113", "username": "ToucanMall", "location": "", "text": "RT @ObamaPalooza: Tiffany Shared What $2,000 Meant to Her ... and the President Stopped by to Talk About It http://t.co/sgT7lsNV #Obama", "timestamp": "Thu Dec 06 16:53:12 PST 2012" }
+, { "id": "nc1:115", "username": "thewildpitch", "location": "", "text": "RT @RevkahJC: Dennis Miller: Obama Should Just Say He Wants To Tax Successful People http://t.co/Ihlemy9Y", "timestamp": "Thu Dec 06 16:53:11 PST 2012" }
+, { "id": "nc1:117", "username": "Rnugent24", "location": "", "text": "RT @ConservativeQuo: unemployment is above 8% again. I wonder how long it will take for Obama to start blaming Bush? 3-2-1 #tcot #antiobama", "timestamp": "Thu Dec 06 16:53:10 PST 2012" }
+ ]
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 575827e..59e76e0 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -6085,29 +6085,47 @@
<output-dir compare="Text">feeds_04</output-dir>
</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>
+
<!--Disable it because of sporadic failures. Raman will re-enable it.
<test-case FilePath="feeds">
<compilation-unit name="feeds_06">
<output-dir compare="Text">feeds_06</output-dir>
</compilation-unit>
</test-case>
- -->
<test-case FilePath="feeds">
<compilation-unit name="feeds_07">
<output-dir compare="Text">feeds_07</output-dir>
</compilation-unit>
</test-case>
+ -->
<test-case FilePath="feeds">
<compilation-unit name="feeds_08">
<output-dir compare="Text">feeds_08</output-dir>
</compilation-unit>
</test-case>
<test-case FilePath="feeds">
+ <compilation-unit name="feeds_09">
+ <output-dir compare="Text">feeds_09</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_10">
+ <output-dir compare="Text">feeds_10</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_11">
+ <output-dir compare="Text">feeds_11</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="feeds">
+ <compilation-unit name="feeds_12">
+ <output-dir compare="Text">feeds_12</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/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 abac9df..b9c96d1 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,10 +37,14 @@
INDEX_DECL,
CREATE_DATAVERSE,
INDEX_DROP,
- CREATE_FEED,
+ CREATE_PRIMARY_FEED,
+ CREATE_SECONDARY_FEED,
DROP_FEED,
CONNECT_FEED,
DISCONNECT_FEED,
+ SUBSCRIBE_FEED,
+ CREATE_FEED_POLICY,
+ DROP_FEED_POLICY,
CREATE_FUNCTION,
FUNCTION_DROP,
COMPACT,
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
index 25acff8..5d9794e 100644
--- 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
@@ -32,9 +32,10 @@
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter.AdapterType;
import edu.uci.ics.asterix.metadata.entities.Feed;
import edu.uci.ics.asterix.metadata.entities.Function;
+import edu.uci.ics.asterix.metadata.entities.PrimaryFeed;
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.metadata.feeds.IFeedAdapterFactory;
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;
@@ -75,6 +76,7 @@
this.varCounter = varCounter;
}
+ /*
public void initialize(MetadataTransactionContext mdTxnCtx, Dataset targetDataset, Feed sourceFeed)
throws MetadataException {
query = new Query();
@@ -91,9 +93,9 @@
}
}
- Triple<IAdapterFactory, ARecordType, AdapterType> factoryOutput = null;
+ Triple<IFeedAdapterFactory, ARecordType, AdapterType> factoryOutput = null;
try {
- factoryOutput = FeedUtil.getFeedFactoryAndOutput(sourceFeed, mdTxnCtx);
+ factoryOutput = FeedUtil.getPrimaryFeedFactoryAndOutput((PrimaryFeed) sourceFeed, mdTxnCtx);
adapterOutputType = factoryOutput.second.getTypeName();
} catch (AlgebricksException ae) {
ae.printStackTrace();
@@ -135,7 +137,7 @@
throw new MetadataException(pe);
}
- }
+ }*/
public Identifier getDataverseName() {
return dataverseName;
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateFeedPolicyStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateFeedPolicyStatement.java
new file mode 100644
index 0000000..939d777
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateFeedPolicyStatement.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.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 CreateFeedPolicyStatement implements Statement {
+
+ private final String policyName;
+ private final String sourcePolicyName;
+ private final Map<String, String> properties;
+ private final String sourcePolicyFile;
+ private final String description;
+ private final boolean ifNotExists;
+
+ public CreateFeedPolicyStatement(String policyName, String sourcePolicyName, Map<String, String> properties,
+ String description, boolean ifNotExists) {
+ this.policyName = policyName;
+ this.sourcePolicyName = sourcePolicyName;
+ this.properties = properties;
+ this.description = description;
+ this.ifNotExists = ifNotExists;
+ sourcePolicyFile = null;
+ }
+
+ public CreateFeedPolicyStatement(String policyName, String sourcePolicyFile, String description, boolean ifNotExists) {
+ this.policyName = policyName;
+ this.sourcePolicyName = null;
+ this.sourcePolicyFile = sourcePolicyFile;
+ this.description = description;
+ this.properties = null;
+ this.ifNotExists = ifNotExists;
+ }
+
+ public boolean getIfNotExists() {
+ return this.ifNotExists;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Statement.Kind.CREATE_FEED_POLICY;
+ }
+
+ @Override
+ public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+ return visitor.visitCreateFeedPolicyStatement(this, arg);
+ }
+
+ @Override
+ public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
+ visitor.visit(this, arg);
+ }
+
+ public String getPolicyName() {
+ return policyName;
+ }
+
+ public String getSourcePolicyName() {
+ return sourcePolicyName;
+ }
+
+ public Map<String, String> getProperties() {
+ return properties;
+ }
+
+ public String getSourcePolicyFile() {
+ return sourcePolicyFile;
+ }
+
+ public String getDescription() {
+ return description;
+ }
+
+}
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
index 4e33f73..e189fd4 100644
--- 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
@@ -14,47 +14,32 @@
*/
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;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-public class CreateFeedStatement implements Statement {
+public abstract class CreateFeedStatement implements Statement {
- private final Identifier dataverseName;
- private final Identifier feedName;
- private final String adapterName;
- private final Map<String, String> adapterConfiguration;
+ private final Pair<Identifier, Identifier> qName;
private final FunctionSignature appliedFunction;
private final boolean ifNotExists;
- public CreateFeedStatement(Identifier dataverseName, Identifier feedName, String adapterName,
- Map<String, String> adapterConfiguration, FunctionSignature appliedFunction, boolean ifNotExists) {
- this.feedName = feedName;
- this.dataverseName = dataverseName;
- this.adapterName = adapterName;
- this.adapterConfiguration = adapterConfiguration;
+ public CreateFeedStatement(Pair<Identifier, Identifier> qName, FunctionSignature appliedFunction,
+ boolean ifNotExists) {
+ this.qName = qName;
this.appliedFunction = appliedFunction;
this.ifNotExists = ifNotExists;
}
public Identifier getDataverseName() {
- return dataverseName;
+ return qName.first;
}
public Identifier getFeedName() {
- return feedName;
- }
-
- public String getAdapterName() {
- return adapterName;
- }
-
- public Map<String, String> getAdapterConfiguration() {
- return adapterConfiguration;
+ return qName.second;
}
public FunctionSignature getAppliedFunction() {
@@ -66,14 +51,10 @@
}
@Override
- public Kind getKind() {
- return Kind.CREATE_FEED;
- }
+ public abstract Kind getKind();
@Override
- public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
- return visitor.visitCreateFeedStatement(this, arg);
- }
+ public abstract <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException;
@Override
public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreatePrimaryFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreatePrimaryFeedStatement.java
new file mode 100644
index 0000000..810e508
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreatePrimaryFeedStatement.java
@@ -0,0 +1,54 @@
+/*
+ * 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.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+
+public class CreatePrimaryFeedStatement extends CreateFeedStatement implements Statement {
+
+ private final String adaptorName;
+ private final Map<String, String> adaptorConfiguration;
+
+ public CreatePrimaryFeedStatement(Pair<Identifier, Identifier> qName, String adaptorName,
+ Map<String, String> adaptorConfiguration, FunctionSignature appliedFunction, boolean ifNotExists) {
+ super(qName, appliedFunction, ifNotExists);
+ this.adaptorName = adaptorName;
+ this.adaptorConfiguration = adaptorConfiguration;
+ }
+
+ public String getAdaptorName() {
+ return adaptorName;
+ }
+
+ public Map<String, String> getAdaptorConfiguration() {
+ return adaptorConfiguration;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.CREATE_PRIMARY_FEED;
+ }
+
+ @Override
+ public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+ return visitor.visitCreatePrimaryFeedStatement(this, arg);
+ }
+}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateSecondaryFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateSecondaryFeedStatement.java
new file mode 100644
index 0000000..7d5f72a
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/CreateSecondaryFeedStatement.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.aql.expression;
+
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+
+/**
+ * Represents the AQL statement for creating a secondary feed.
+ * A secondary feed is one that derives its data from another (primary/secondary) feed.
+ */
+public class CreateSecondaryFeedStatement extends CreateFeedStatement implements Statement {
+
+ /** The source feed that provides data for this secondary feed. */
+ private final Pair<Identifier, Identifier> sourceQName;
+
+ public CreateSecondaryFeedStatement(Pair<Identifier, Identifier> qName, Pair<Identifier, Identifier> sourceQName,
+ FunctionSignature appliedFunction, boolean ifNotExists) {
+ super(qName, appliedFunction, ifNotExists);
+ this.sourceQName = sourceQName;
+ }
+
+ public String getSourceFeedDataverse() {
+ return sourceQName.first != null ? sourceQName.first.toString()
+ : getDataverseName() != null ? getDataverseName().getValue() : null;
+ }
+
+ public String getSourceFeedName() {
+ return sourceQName.second != null ? sourceQName.second.toString() : null;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.CREATE_SECONDARY_FEED;
+ }
+
+ @Override
+ public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+ return visitor.visitCreateSecondaryFeedStatement(this, arg);
+ }
+
+}
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedPolicyDropStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedPolicyDropStatement.java
new file mode 100644
index 0000000..a9e9ee1
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/FeedPolicyDropStatement.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 FeedPolicyDropStatement implements Statement {
+
+ private final Identifier dataverseName;
+ private final Identifier policyName;
+ private boolean ifExists;
+
+ public FeedPolicyDropStatement(Identifier dataverseName, Identifier policyName, boolean ifExists) {
+ this.dataverseName = dataverseName;
+ this.policyName = policyName;
+ this.ifExists = ifExists;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.DROP_FEED_POLICY;
+ }
+
+ public Identifier getDataverseName() {
+ return dataverseName;
+ }
+
+ public Identifier getPolicyName() {
+ return policyName;
+ }
+
+ public boolean getIfExists() {
+ return ifExists;
+ }
+
+ @Override
+ public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+ return visitor.visitDropFeedPolicyStatement(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/SubscribeFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/SubscribeFeedStatement.java
new file mode 100644
index 0000000..509a69a
--- /dev/null
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/SubscribeFeedStatement.java
@@ -0,0 +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.
+ */
+package edu.uci.ics.asterix.aql.expression;
+
+import java.io.StringReader;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+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.feeds.FeedActivity;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionRequest;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+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.DatasourceAdapter.AdapterType;
+import edu.uci.ics.asterix.metadata.entities.Feed;
+import edu.uci.ics.asterix.metadata.entities.Function;
+import edu.uci.ics.asterix.metadata.entities.PrimaryFeed;
+import edu.uci.ics.asterix.metadata.entities.SecondaryFeed;
+import edu.uci.ics.asterix.metadata.feeds.FeedUtil;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapterFactory;
+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.Triple;
+
+/**
+ * Represents the AQL statement for subscribing to a feed.
+ * This AQL statement is private and may not be used by the end-user.
+ */
+public class SubscribeFeedStatement implements Statement {
+
+ private static final Logger LOGGER = Logger.getLogger(SubscribeFeedStatement.class.getName());
+ private final FeedConnectionRequest connectionRequest;
+ private Query query;
+ private int varCounter;
+ private final String[] locations;
+
+ public static final String WAIT_FOR_COMPLETION = "wait-for-completion-feed";
+
+ public SubscribeFeedStatement(String[] locations, FeedConnectionRequest subscriptionRequest) {
+ this.connectionRequest = subscriptionRequest;
+ this.varCounter = 0;
+ this.locations = locations;
+ }
+
+ public void initialize(MetadataTransactionContext mdTxnCtx) throws MetadataException {
+ this.query = new Query();
+ FeedId sourceFeedId = connectionRequest.getFeedJointKey().getFeedId();
+ Feed subscriberFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, connectionRequest.getReceivingFeedId()
+ .getDataverse(), connectionRequest.getReceivingFeedId().getFeedName());
+ if (subscriberFeed == null) {
+ throw new IllegalStateException(" Subscriber feed " + subscriberFeed + " not found.");
+ }
+
+ String feedOutputType = getOutputType(mdTxnCtx);
+ FunctionSignature appliedFunction = subscriberFeed.getAppliedFunction();
+ Function function = 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");
+ }
+ }
+
+ StringBuilder builder = new StringBuilder();
+ builder.append("use dataverse " + sourceFeedId.getDataverse() + ";\n");
+ builder.append("set" + " " + FunctionUtils.IMPORT_PRIVATE_FUNCTIONS + " " + "'" + Boolean.TRUE + "'" + ";\n");
+ builder.append("set" + " " + FeedActivity.FeedActivityDetails.FEED_POLICY_NAME + " " + "'"
+ + connectionRequest.getPolicy() + "'" + ";\n");
+
+ builder.append("insert into dataset " + connectionRequest.getTargetDataset() + " ");
+ builder.append(" (" + " for $x in feed-collect ('" + sourceFeedId.getDataverse() + "'" + "," + "'"
+ + sourceFeedId.getFeedName() + "'" + "," + "'" + connectionRequest.getReceivingFeedId().getFeedName()
+ + "'" + "," + "'" + connectionRequest.getSubscriptionLocation().name() + "'" + "," + "'"
+ + connectionRequest.getTargetDataset() + "'" + "," + "'" + feedOutputType + "'" + ")");
+
+ List<String> functionsToApply = connectionRequest.getFunctionsToApply();
+ if (functionsToApply != null && functionsToApply.isEmpty()) {
+ builder.append(" return $x");
+ } else {
+ String rValueName = "x";
+ String lValueName = "y";
+ int variableIndex = 0;
+ for (String functionName : functionsToApply) {
+ function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, appliedFunction);
+ variableIndex++;
+ switch (function.getLanguage().toUpperCase()) {
+ case Function.LANGUAGE_AQL:
+ builder.append(" let " + "$" + lValueName + variableIndex + ":=(" + function.getFunctionBody()
+ + ")");
+ builder.append("\n");
+ break;
+ case Function.LANGUAGE_JAVA:
+ builder.append(" let " + "$" + lValueName + variableIndex + ":=" + functionName + "(" + "$"
+ + rValueName + ")");
+ rValueName = lValueName + variableIndex;
+ break;
+ }
+ builder.append("\n");
+ }
+ builder.append("return $" + lValueName + variableIndex);
+ }
+ builder.append(")");
+ builder.append(";");
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Connect feed statement translated to\n" + builder.toString());
+ }
+ AQLParser parser = new AQLParser(new StringReader(builder.toString()));
+
+ List<Statement> statements;
+ try {
+ statements = parser.Statement();
+ query = ((InsertStatement) statements.get(3)).getQuery();
+ } catch (ParseException pe) {
+ throw new MetadataException(pe);
+ }
+
+ }
+
+ public Query getQuery() {
+ return query;
+ }
+
+ public int getVarCounter() {
+ return varCounter;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.SUBSCRIBE_FEED;
+ }
+
+ public String getPolicy() {
+ return connectionRequest.getPolicy();
+ }
+
+ public FeedConnectionRequest getSubscriptionRequest() {
+ return connectionRequest;
+ }
+
+ @Override
+ public <R, T> R accept(IAqlExpressionVisitor<R, T> visitor, T arg) throws AsterixException {
+ return null;
+ }
+
+ @Override
+ public <T> void accept(IAqlVisitorWithVoidReturn<T> visitor, T arg) throws AsterixException {
+ }
+
+ public String getDataverseName() {
+ return connectionRequest.getReceivingFeedId().getDataverse();
+ }
+
+ private String getOutputType(MetadataTransactionContext mdTxnCtx) throws MetadataException {
+ String outputType = null;
+ FeedId feedId = connectionRequest.getReceivingFeedId();
+ Feed feed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getFeedName());
+ FeedPolicyAccessor policyAccessor = new FeedPolicyAccessor(connectionRequest.getPolicyParameters());
+ try {
+ switch (feed.getFeedType()) {
+ case PRIMARY:
+ Triple<IFeedAdapterFactory, ARecordType, AdapterType> factoryOutput = null;
+
+ factoryOutput = FeedUtil.getPrimaryFeedFactoryAndOutput((PrimaryFeed) feed, policyAccessor,
+ mdTxnCtx);
+ outputType = factoryOutput.second.getTypeName();
+ break;
+ case SECONDARY:
+ outputType = FeedUtil.getSecondaryFeedOutput((SecondaryFeed) feed, policyAccessor, mdTxnCtx);
+ break;
+ }
+ return outputType;
+
+ } catch (AlgebricksException ae) {
+ throw new MetadataException(ae);
+ }
+ }
+
+ public String[] getLocations() {
+ return locations;
+ }
+}
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 21a2388..28acfda 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
@@ -25,6 +25,7 @@
import edu.uci.ics.asterix.aql.expression.CompactStatement;
import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedPolicyStatement;
import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
@@ -37,6 +38,7 @@
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.FeedPolicyDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -61,13 +63,13 @@
import edu.uci.ics.asterix.aql.expression.OrderbyClause;
import edu.uci.ics.asterix.aql.expression.OrderbyClause.OrderModifier;
import edu.uci.ics.asterix.aql.expression.OrderedListTypeDefinition;
-import edu.uci.ics.asterix.aql.expression.RunStatement;
import edu.uci.ics.asterix.aql.expression.QuantifiedExpression;
import edu.uci.ics.asterix.aql.expression.QuantifiedPair;
import edu.uci.ics.asterix.aql.expression.Query;
import edu.uci.ics.asterix.aql.expression.RecordConstructor;
import edu.uci.ics.asterix.aql.expression.RecordTypeDefinition;
import edu.uci.ics.asterix.aql.expression.RecordTypeDefinition.RecordKind;
+import edu.uci.ics.asterix.aql.expression.RunStatement;
import edu.uci.ics.asterix.aql.expression.SetStatement;
import edu.uci.ics.asterix.aql.expression.TypeDecl;
import edu.uci.ics.asterix.aql.expression.TypeDropStatement;
@@ -557,6 +559,15 @@
// TODO Auto-generated method stub
}
+
+ @Override
+ public void visit(CreateFeedPolicyStatement stmt, Integer arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ }
+
+ @Override
+ public void visit(FeedPolicyDropStatement stmt, Integer arg) throws AsterixException {
+ }
@Override
public void visit(RunStatement stmt, Integer arg) throws AsterixException {
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 84d9726..6716fff 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
@@ -15,21 +15,24 @@
package edu.uci.ics.asterix.aql.expression.visitor;
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.ConnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
-import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedPolicyStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
+import edu.uci.ics.asterix.aql.expression.CreatePrimaryFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateSecondaryFeedStatement;
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.FeedDropStatement;
+import edu.uci.ics.asterix.aql.expression.FeedPolicyDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.ForClause;
import edu.uci.ics.asterix.aql.expression.FunctionDecl;
@@ -157,9 +160,15 @@
R visitConnectFeedStatement(ConnectFeedStatement del, T arg) throws AsterixException;
- R visitCreateFeedStatement(CreateFeedStatement del, T arg) throws AsterixException;
+ R visitCreatePrimaryFeedStatement(CreatePrimaryFeedStatement cpfs, T arg) throws AsterixException;
+
+ R visitCreateSecondaryFeedStatement(CreateSecondaryFeedStatement csfs, T arg) throws AsterixException;
R visitDropFeedStatement(FeedDropStatement del, T arg) throws AsterixException;
+
+ R visitDropFeedPolicyStatement(FeedPolicyDropStatement dfs, T arg) throws AsterixException;
+
+ R visitCreateFeedPolicyStatement(CreateFeedPolicyStatement cfps, T arg) throws AsterixException;
R visitCallExpr(CallExpr pf, 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 f04c3a0..84e0825 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
@@ -18,6 +18,7 @@
import edu.uci.ics.asterix.aql.expression.CompactStatement;
import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedPolicyStatement;
import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
@@ -30,6 +31,7 @@
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.FeedPolicyDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.ForClause;
import edu.uci.ics.asterix.aql.expression.FunctionDecl;
@@ -49,11 +51,11 @@
import edu.uci.ics.asterix.aql.expression.OperatorExpr;
import edu.uci.ics.asterix.aql.expression.OrderbyClause;
import edu.uci.ics.asterix.aql.expression.OrderedListTypeDefinition;
-import edu.uci.ics.asterix.aql.expression.RunStatement;
import edu.uci.ics.asterix.aql.expression.QuantifiedExpression;
import edu.uci.ics.asterix.aql.expression.Query;
import edu.uci.ics.asterix.aql.expression.RecordConstructor;
import edu.uci.ics.asterix.aql.expression.RecordTypeDefinition;
+import edu.uci.ics.asterix.aql.expression.RunStatement;
import edu.uci.ics.asterix.aql.expression.SetStatement;
import edu.uci.ics.asterix.aql.expression.TypeDecl;
import edu.uci.ics.asterix.aql.expression.TypeDropStatement;
@@ -164,8 +166,12 @@
void visit(CreateFeedStatement stmt, T arg) throws AsterixException;
+ void visit(CreateFeedPolicyStatement stmt, T arg) throws AsterixException;
+
void visit(FeedDropStatement stmt, T arg) throws AsterixException;
+ void visit(FeedPolicyDropStatement stmt, T arg) throws AsterixException;
+
void visit(CreateFunctionStatement cfs, T arg) throws AsterixException;
void visit(FunctionDropStatement fds, 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 c416fa6..6b1df98 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
@@ -25,21 +25,24 @@
import edu.uci.ics.asterix.aql.base.Expression;
import edu.uci.ics.asterix.aql.base.Expression.Kind;
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.ConnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
-import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedPolicyStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
+import edu.uci.ics.asterix.aql.expression.CreatePrimaryFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateSecondaryFeedStatement;
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.FeedPolicyDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -89,8 +92,6 @@
import edu.uci.ics.asterix.metadata.entities.Function;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
import edu.uci.ics.asterix.om.functions.AsterixFunction;
-import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
-import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
public final class AqlRewriter {
@@ -553,8 +554,13 @@
}
@Override
- public Void visitCreateFeedStatement(CreateFeedStatement del, Void arg) throws AsterixException {
- // TODO Auto-generated method stub
+ public Void visitCreatePrimaryFeedStatement(CreatePrimaryFeedStatement del, Void arg) throws AsterixException {
+ return null;
+ }
+
+ @Override
+ public Void visitCreateSecondaryFeedStatement(CreateSecondaryFeedStatement del, Void arg)
+ throws AsterixException {
return null;
}
@@ -575,6 +581,18 @@
// TODO Auto-generated method stub
return null;
}
+
+ @Override
+ public Void visitCreateFeedPolicyStatement(CreateFeedPolicyStatement cfps, Void arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Void visitDropFeedPolicyStatement(FeedPolicyDropStatement dfs, 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 edb080c..f03799a 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
@@ -22,21 +22,24 @@
import edu.uci.ics.asterix.aql.base.Expression;
import edu.uci.ics.asterix.aql.base.IAqlExpression;
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.ConnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
-import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedPolicyStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
+import edu.uci.ics.asterix.aql.expression.CreatePrimaryFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateSecondaryFeedStatement;
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.FeedPolicyDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -588,8 +591,15 @@
}
@Override
- public Pair<IAqlExpression, List<VariableSubstitution>> visitCreateFeedStatement(CreateFeedStatement del,
- List<VariableSubstitution> arg) throws AsterixException {
+ public Pair<IAqlExpression, List<VariableSubstitution>> visitCreatePrimaryFeedStatement(
+ CreatePrimaryFeedStatement del, List<VariableSubstitution> arg) throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Pair<IAqlExpression, List<VariableSubstitution>> visitCreateSecondaryFeedStatement(
+ CreateSecondaryFeedStatement del, List<VariableSubstitution> arg) throws AsterixException {
// TODO Auto-generated method stub
return null;
}
@@ -614,4 +624,17 @@
// TODO Auto-generated method stub
return null;
}
+
+
+ @Override
+ public Pair<IAqlExpression, List<VariableSubstitution>> visitCreateFeedPolicyStatement(
+ CreateFeedPolicyStatement cfps, List<VariableSubstitution> arg) throws AsterixException {
+ return null;
+ }
+
+ @Override
+ public Pair<IAqlExpression, List<VariableSubstitution>> visitDropFeedPolicyStatement(FeedPolicyDropStatement dfs,
+ List<VariableSubstitution> arg) throws AsterixException {
+ return null;
+ }
}
\ No newline at end of file
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 58433d3..394b507 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
@@ -23,21 +23,24 @@
import edu.uci.ics.asterix.aql.base.Expression.Kind;
import edu.uci.ics.asterix.aql.base.IAqlExpression;
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.ConnectFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
-import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedPolicyStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
+import edu.uci.ics.asterix.aql.expression.CreatePrimaryFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateSecondaryFeedStatement;
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.FeedPolicyDropStatement;
import edu.uci.ics.asterix.aql.expression.FieldAccessor;
import edu.uci.ics.asterix.aql.expression.FieldBinding;
import edu.uci.ics.asterix.aql.expression.ForClause;
@@ -522,7 +525,15 @@
}
@Override
- public Boolean visitCreateFeedStatement(CreateFeedStatement del, List<FunctionDecl> arg) throws AsterixException {
+ public Boolean visitCreatePrimaryFeedStatement(CreatePrimaryFeedStatement del, List<FunctionDecl> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Boolean visitCreateSecondaryFeedStatement(CreateSecondaryFeedStatement del, List<FunctionDecl> arg)
+ throws AsterixException {
// TODO Auto-generated method stub
return null;
}
@@ -544,4 +555,18 @@
// TODO Auto-generated method stub
return null;
}
+
+ @Override
+ public Boolean visitCreateFeedPolicyStatement(CreateFeedPolicyStatement cfps, List<FunctionDecl> arg)
+ throws AsterixException {
+ // TODO Auto-generated method stub
+ return null;
+ }
+
+ @Override
+ public Boolean visitDropFeedPolicyStatement(FeedPolicyDropStatement dfs, 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 8150871..7044962 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -37,6 +37,9 @@
import edu.uci.ics.asterix.aql.expression.CallExpr;
import edu.uci.ics.asterix.aql.expression.CompactStatement;
import edu.uci.ics.asterix.aql.expression.ConnectFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateFeedPolicyStatement;
+import edu.uci.ics.asterix.aql.expression.CreatePrimaryFeedStatement;
+import edu.uci.ics.asterix.aql.expression.CreateSecondaryFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
import edu.uci.ics.asterix.aql.expression.CreateFeedStatement;
import edu.uci.ics.asterix.aql.expression.CreateFunctionStatement;
@@ -336,6 +339,7 @@
| stmt = DataverseSpecification()
| stmt = FunctionSpecification()
| stmt = FeedSpecification()
+ | stmt = FeedPolicySpecification()
)
{
return stmt;
@@ -653,24 +657,61 @@
Map<String,String> properties = null;
FunctionSignature appliedFunction = null;
CreateFeedStatement cfs = null;
+ Pair<Identifier,Identifier> sourceNameComponents = null;
+
}
{
(
- "feed" nameComponents = QualifiedName()
- ifNotExists = IfNotExists()
- "using" adapterName = AdapterName() properties = Configuration()
- (appliedFunction = ApplyFunction())?
+ "secondary" "feed" nameComponents = QualifiedName() ifNotExists = IfNotExists()
+ <FROM> "feed" sourceNameComponents = QualifiedName() (appliedFunction = ApplyFunction())?
{
- cfs = new CreateFeedStatement(nameComponents.first,
- nameComponents.second, adapterName, properties, appliedFunction, ifNotExists);
+ cfs = new CreateSecondaryFeedStatement(nameComponents,
+ sourceNameComponents, appliedFunction, ifNotExists);
}
-
+ |
+ ("primary")? "feed" nameComponents = QualifiedName() ifNotExists = IfNotExists()
+ "using" adapterName = AdapterName() properties = Configuration() (appliedFunction = ApplyFunction())?
+ {
+ cfs = new CreatePrimaryFeedStatement(nameComponents,
+ adapterName, properties, appliedFunction, ifNotExists);
+ }
)
{
return cfs;
}
}
+CreateFeedPolicyStatement FeedPolicySpecification() throws ParseException:
+{
+ String policyName = null;
+ String basePolicyName = null;
+ String sourcePolicyFile = null;
+ String definition = null;
+ boolean ifNotExists = false;
+ Map<String,String> properties = null;
+ CreateFeedPolicyStatement cfps = null;
+}
+{
+ (
+ "ingestion" "policy" policyName = Identifier() ifNotExists = IfNotExists()
+ <FROM>
+ ("policy" basePolicyName = Identifier() properties = Configuration() ("definition" definition = StringLiteral())?
+ {
+ cfps = new CreateFeedPolicyStatement(policyName,
+ basePolicyName, properties, definition, ifNotExists);
+ }
+ | "path" sourcePolicyFile = Identifier() ("definition" definition = StringLiteral())?
+ {
+ cfps = new CreateFeedPolicyStatement(policyName, sourcePolicyFile, definition, ifNotExists);
+ }
+ )
+
+ )
+ {
+ return cfps;
+ }
+}
+
List<VarIdentifier> ParameterList() throws ParseException:
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
index 8059ede..73c61f0 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
@@ -20,7 +20,7 @@
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.feeds.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.IIOManager;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterEventsSubscriber.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IClusterEventsSubscriber.java
similarity index 81%
rename from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterEventsSubscriber.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IClusterEventsSubscriber.java
index 049a45c..23b5fd2 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterEventsSubscriber.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IClusterEventsSubscriber.java
@@ -1,4 +1,4 @@
-package edu.uci.ics.asterix.metadata.api;
+package edu.uci.ics.asterix.common.api;
/*
* Copyright 2009-2013 by The Regents of the University of California
@@ -16,8 +16,7 @@
*/
import java.util.Set;
-import edu.uci.ics.asterix.metadata.cluster.IClusterManagementWorkResponse;
-import edu.uci.ics.asterix.om.util.AsterixClusterProperties.State;
+import edu.uci.ics.asterix.common.api.IClusterManagementWork.ClusterState;
public interface IClusterEventsSubscriber {
@@ -42,6 +41,7 @@
* @param previousState
* @param newState
*/
- public void notifyStateChange(State previousState, State newState);
+ public void notifyStateChange(ClusterState previousState, ClusterState newState);
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IClusterManagementWork.java
similarity index 88%
rename from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IClusterManagementWork.java
index 65ac354..dc7a69c 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IClusterManagementWork.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.metadata.api;
+package edu.uci.ics.asterix.common.api;
public interface IClusterManagementWork {
@@ -21,6 +21,11 @@
REMOVE_NODE
}
+ public enum ClusterState {
+ ACTIVE,
+ UNUSABLE
+ }
+
public WorkType getClusterManagementWorkType();
public int getWorkId();
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/IClusterManagementWorkResponse.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IClusterManagementWorkResponse.java
similarity index 76%
rename from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/IClusterManagementWorkResponse.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IClusterManagementWorkResponse.java
index dfc88ac..25fcda0 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/cluster/IClusterManagementWorkResponse.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IClusterManagementWorkResponse.java
@@ -1,6 +1,5 @@
-package edu.uci.ics.asterix.metadata.cluster;
+package edu.uci.ics.asterix.common.api;
-import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
public interface IClusterManagementWorkResponse {
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixFeedProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixFeedProperties.java
new file mode 100644
index 0000000..aa114e6
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixFeedProperties.java
@@ -0,0 +1,62 @@
+/*
+ * 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.common.config;
+
+public class AsterixFeedProperties extends AbstractAsterixProperties {
+
+ private static final String FEED_CENTRAL_MANAGER_PORT_KEY = "feed.central.manager.port";
+ private static final int FEED_CENTRAL_MANAGER_PORT_DEFAULT = 4500; // port at which the Central Feed Manager listens for control messages from local Feed Managers
+
+ private static final String FEED_MEMORY_GLOBALBUDGET_KEY = "feed.memory.global.budget";
+ private static final long FEED_MEMORY_GLOBALBUDGET_DEFAULT = 67108864; // 64MB or 2048 frames (assuming 32768 as frame size)
+
+ private static final String FEED_MEMORY_AVAILABLE_WAIT_TIMEOUT_KEY = "feed.memory.available.wait.timeout";
+ private static final long FEED_MEMORY_AVAILABLE_WAIT_TIMEOUT_DEFAULT = 10; // 10 seconds
+
+ private static final String FEED_PENDING_WORK_THRESHOLD_KEY = "feed.pending.work.threshold";
+ private static final int FEED_PENDING_WORK_THRESHOLD_DEFAULT = 50; // maximum length of input queue before triggering corrective action
+
+ private static final String FEED_MAX_SUCCESSIVE_THRESHOLD_PERIOD_KEY = "feed.max.threshold.period";
+ private static final int FEED_MAX_SUCCESSIVE_THRESHOLD_PERIOD_DEFAULT = 5;
+
+ public AsterixFeedProperties(AsterixPropertiesAccessor accessor) {
+ super(accessor);
+ }
+
+ public long getMemoryComponentGlobalBudget() {
+ return accessor.getProperty(FEED_MEMORY_GLOBALBUDGET_KEY, FEED_MEMORY_GLOBALBUDGET_DEFAULT,
+ PropertyInterpreters.getLongPropertyInterpreter());
+ }
+
+ public long getMemoryAvailableWaitTimeout() {
+ return accessor.getProperty(FEED_MEMORY_AVAILABLE_WAIT_TIMEOUT_KEY, FEED_MEMORY_AVAILABLE_WAIT_TIMEOUT_DEFAULT,
+ PropertyInterpreters.getLongPropertyInterpreter());
+ }
+
+ public int getFeedCentralManagerPort() {
+ return accessor.getProperty(FEED_CENTRAL_MANAGER_PORT_KEY, FEED_CENTRAL_MANAGER_PORT_DEFAULT,
+ PropertyInterpreters.getIntegerPropertyInterpreter());
+ }
+
+ public int getPendingWorkThreshold() {
+ return accessor.getProperty(FEED_PENDING_WORK_THRESHOLD_KEY, FEED_PENDING_WORK_THRESHOLD_DEFAULT,
+ PropertyInterpreters.getIntegerPropertyInterpreter());
+ }
+
+ public int getMaxSuccessiveThresholdPeriod() {
+ return accessor.getProperty(FEED_MAX_SUCCESSIVE_THRESHOLD_PERIOD_KEY,
+ FEED_MAX_SUCCESSIVE_THRESHOLD_PERIOD_DEFAULT, PropertyInterpreters.getIntegerPropertyInterpreter());
+ }
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/IAsterixPropertiesProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/IAsterixPropertiesProvider.java
index e696519..32386f6 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/IAsterixPropertiesProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/IAsterixPropertiesProvider.java
@@ -24,4 +24,6 @@
public AsterixMetadataProperties getMetadataProperties();
public AsterixExternalProperties getExternalProperties();
+
+ public AsterixFeedProperties getFeedProperties();
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/exceptions/FrameDataException.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/exceptions/FrameDataException.java
new file mode 100644
index 0000000..4651607
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/exceptions/FrameDataException.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.common.exceptions;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class FrameDataException extends HyracksDataException {
+
+ private static final long serialVersionUID = 1L;
+
+ private final int tupleIndex;
+
+ public FrameDataException(int tupleIndex, Exception cause) {
+ super(cause);
+ this.tupleIndex = tupleIndex;
+ }
+
+ public int getTupleIndex() {
+ return tupleIndex;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/BasicMonitoredBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/BasicMonitoredBuffer.java
new file mode 100644
index 0000000..ba38f87
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/BasicMonitoredBuffer.java
@@ -0,0 +1,56 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import edu.uci.ics.asterix.common.feeds.api.IExceptionHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector;
+import edu.uci.ics.asterix.common.feeds.api.IFrameEventCallback;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class BasicMonitoredBuffer extends MonitoredBuffer {
+
+ public BasicMonitoredBuffer(IHyracksTaskContext ctx, FeedRuntimeInputHandler inputHandler, IFrameWriter frameWriter, FrameTupleAccessor fta,
+ RecordDescriptor recordDesc, IFeedMetricCollector metricCollector,
+ FeedConnectionId connectionId, FeedRuntimeId runtimeId, IExceptionHandler exceptionHandler,
+ IFrameEventCallback callback, int nPartitions, FeedPolicyAccessor policyAccessor) {
+ super(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector, connectionId, runtimeId,
+ exceptionHandler, callback, nPartitions, policyAccessor);
+ }
+
+ @Override
+ protected boolean monitorProcessingRate() {
+ return false;
+ }
+
+ @Override
+ protected boolean logInflowOutflowRate() {
+ return false;
+ }
+
+ @Override
+ protected IFramePreprocessor getFramePreProcessor() {
+ return null;
+ }
+
+ @Override
+ protected IFramePostProcessor getFramePostProcessor() {
+ return null;
+ }
+
+ @Override
+ protected boolean monitorInputQueueLength() {
+ return false;
+ }
+
+ @Override
+ protected boolean reportInflowRate() {
+ return false;
+ }
+
+ @Override
+ protected boolean reportOutflowRate() {
+ return false;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/CollectionRuntime.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/CollectionRuntime.java
new file mode 100644
index 0000000..e324617
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/CollectionRuntime.java
@@ -0,0 +1,88 @@
+/*
+ * 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.common.feeds;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.feeds.FeedFrameCollector.State;
+import edu.uci.ics.asterix.common.feeds.api.ISubscribableRuntime;
+import edu.uci.ics.asterix.common.feeds.api.ISubscriberRuntime;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+/**
+ * Represents the feed runtime that collects feed tuples from another feed.
+ * In case of a primary feed, the CollectionRuntime collects tuples from the feed
+ * intake job. For a secondary feed, tuples are collected from the intake/compute
+ * runtime associated with the source feed.
+ */
+public class CollectionRuntime extends FeedRuntime implements ISubscriberRuntime {
+
+ private final FeedConnectionId connectionId;
+ private final ISubscribableRuntime sourceRuntime;
+ private final Map<String, String> feedPolicy;
+ private FeedFrameCollector frameCollector;
+
+ public CollectionRuntime(FeedConnectionId connectionId, FeedRuntimeId runtimeId,
+ FeedRuntimeInputHandler inputSideHandler, IFrameWriter outputSideWriter,
+ ISubscribableRuntime sourceRuntime, Map<String, String> feedPolicy) {
+ super(runtimeId, inputSideHandler, outputSideWriter);
+ this.connectionId = connectionId;
+ this.sourceRuntime = sourceRuntime;
+ this.feedPolicy = feedPolicy;
+ }
+
+ public State waitTillCollectionOver() throws InterruptedException {
+ if (!(isCollectionOver())) {
+ synchronized (frameCollector) {
+ while (!isCollectionOver()) {
+ frameCollector.wait();
+ }
+ }
+ }
+ return frameCollector.getState();
+ }
+
+ private boolean isCollectionOver() {
+ return frameCollector.getState().equals(FeedFrameCollector.State.FINISHED)
+ || frameCollector.getState().equals(FeedFrameCollector.State.HANDOVER);
+ }
+
+ public void setMode(Mode mode) {
+ getInputHandler().setMode(mode);
+ }
+
+ @Override
+ public Map<String, String> getFeedPolicy() {
+ return feedPolicy;
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public ISubscribableRuntime getSourceRuntime() {
+ return sourceRuntime;
+ }
+
+ public void setFrameCollector(FeedFrameCollector frameCollector) {
+ this.frameCollector = frameCollector;
+ }
+
+ public FeedFrameCollector getFrameCollector() {
+ return frameCollector;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/ComputeSideMonitoredBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/ComputeSideMonitoredBuffer.java
new file mode 100644
index 0000000..161f27a
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/ComputeSideMonitoredBuffer.java
@@ -0,0 +1,55 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import edu.uci.ics.asterix.common.feeds.api.IExceptionHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector;
+import edu.uci.ics.asterix.common.feeds.api.IFrameEventCallback;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class ComputeSideMonitoredBuffer extends MonitoredBuffer {
+
+ public ComputeSideMonitoredBuffer(IHyracksTaskContext ctx, FeedRuntimeInputHandler inputHandler, IFrameWriter frameWriter,
+ FrameTupleAccessor fta, RecordDescriptor recordDesc, IFeedMetricCollector metricCollector,
+ FeedConnectionId connectionId, FeedRuntimeId runtimeId, IExceptionHandler exceptionHandler,
+ IFrameEventCallback callback, int nPartitions, FeedPolicyAccessor policyAccessor) {
+ super(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector, connectionId, runtimeId,
+ exceptionHandler, callback, nPartitions, policyAccessor);
+ }
+
+ @Override
+ protected boolean monitorProcessingRate() {
+ return true;
+ }
+
+ protected boolean logInflowOutflowRate() {
+ return true;
+ }
+
+ @Override
+ protected boolean monitorInputQueueLength() {
+ return true;
+ }
+
+ @Override
+ protected IFramePreprocessor getFramePreProcessor() {
+ return null;
+ }
+
+ @Override
+ protected IFramePostProcessor getFramePostProcessor() {
+ return null;
+ }
+
+ @Override
+ protected boolean reportOutflowRate() {
+ return false;
+ }
+
+ @Override
+ protected boolean reportInflowRate() {
+ return false;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/DataBucket.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/DataBucket.java
new file mode 100644
index 0000000..ea4480e
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/DataBucket.java
@@ -0,0 +1,84 @@
+/*
+ * 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.common.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class DataBucket {
+
+ private static final AtomicInteger globalBucketId = new AtomicInteger(0);
+
+ private final ByteBuffer content;
+ private final AtomicInteger readCount;
+ private final int bucketId;
+
+ private int desiredReadCount;
+ private ContentType contentType;
+
+ private final DataBucketPool pool;
+
+ public enum ContentType {
+ DATA, // data (feed tuple)
+ EOD, // A signal indicating that there shall be no more data
+ EOSD // End of processing of spilled data
+ }
+
+ public DataBucket(DataBucketPool pool) {
+ this.content = ByteBuffer.allocate(pool.getFrameSize());
+ this.readCount = new AtomicInteger(0);
+ this.pool = pool;
+ this.contentType = ContentType.DATA;
+ this.bucketId = globalBucketId.incrementAndGet();
+ }
+
+ public synchronized void reset(ByteBuffer frame) {
+ if (frame != null) {
+ content.flip();
+ System.arraycopy(frame.array(), 0, content.array(), 0, frame.limit());
+ content.limit(frame.limit());
+ content.position(0);
+ }
+ }
+
+ public synchronized void doneReading() {
+ if (readCount.incrementAndGet() == desiredReadCount) {
+ readCount.set(0);
+ pool.returnDataBucket(this);
+ }
+ }
+
+ public void setDesiredReadCount(int rCount) {
+ this.desiredReadCount = rCount;
+ }
+
+ public ContentType getContentType() {
+ return contentType;
+ }
+
+ public void setContentType(ContentType contentType) {
+ this.contentType = contentType;
+ }
+
+ public synchronized ByteBuffer getContent() {
+ return content;
+ }
+
+ @Override
+ public String toString() {
+ return "DataBucket [" + bucketId + "]" + " (" + readCount + "," + desiredReadCount + ")";
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/DataBucketPool.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/DataBucketPool.java
new file mode 100644
index 0000000..1a66b38
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/DataBucketPool.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.common.feeds;
+
+import java.util.Stack;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryComponent;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryManager;
+
+/**
+ * Represents a pool of reusable {@link DataBucket}
+ */
+public class DataBucketPool implements IFeedMemoryComponent {
+
+ /** A unique identifier for the memory component **/
+ private final int componentId;
+
+ /** The {@link IFeedMemoryManager} for the NodeController **/
+ private final IFeedMemoryManager memoryManager;
+
+ /** A collection of available data buckets {@link DataBucket} **/
+ private final Stack<DataBucket> pool;
+
+ /** The total number of data buckets {@link DataBucket} allocated **/
+ private int totalAllocation;
+
+ /** The fixed frame size as configured for the asterix runtime **/
+ private final int frameSize;
+
+ public DataBucketPool(int componentId, IFeedMemoryManager memoryManager, int size, int frameSize) {
+ this.componentId = componentId;
+ this.memoryManager = memoryManager;
+ this.pool = new Stack<DataBucket>();
+ this.frameSize = frameSize;
+ expand(size);
+ }
+
+ public synchronized void returnDataBucket(DataBucket bucket) {
+ pool.push(bucket);
+ }
+
+ public synchronized DataBucket getDataBucket() {
+ if (pool.size() == 0) {
+ if (!memoryManager.expandMemoryComponent(this)) {
+ return null;
+ }
+ }
+ return pool.pop();
+ }
+
+ @Override
+ public Type getType() {
+ return Type.POOL;
+ }
+
+ @Override
+ public int getTotalAllocation() {
+ return totalAllocation;
+ }
+
+ @Override
+ public int getComponentId() {
+ return componentId;
+ }
+
+ @Override
+ public void expand(int delta) {
+ for (int i = 0; i < delta; i++) {
+ DataBucket bucket = new DataBucket(this);
+ pool.add(bucket);
+ }
+ totalAllocation += delta;
+ }
+
+ @Override
+ public void reset() {
+ totalAllocation -= pool.size();
+ pool.clear();
+ }
+
+ @Override
+ public String toString() {
+ return "DataBucketPool" + "[" + componentId + "]" + "(" + totalAllocation + ")";
+ }
+
+ public int getSize() {
+ return pool.size();
+ }
+
+ public int getFrameSize() {
+ return frameSize;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/DistributeFeedFrameWriter.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/DistributeFeedFrameWriter.java
new file mode 100644
index 0000000..1e1baca
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/DistributeFeedFrameWriter.java
@@ -0,0 +1,135 @@
+/*
+ * 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.common.feeds;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedOperatorOutputSideHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedOperatorOutputSideHandler.Type;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+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.FrameTupleAccessor;
+
+/**
+ * Provides mechanism for distributing the frames, as received from an operator to a
+ * set of registered readers. Each reader typically operates at a different pace. Readers
+ * are isolated from each other to ensure that a slow reader does not impact the progress of
+ * others.
+ **/
+public class DistributeFeedFrameWriter implements IFrameWriter {
+
+ private static final Logger LOGGER = Logger.getLogger(DistributeFeedFrameWriter.class.getName());
+
+ /** A unique identifier for the feed to which the incoming tuples belong. **/
+ private final FeedId feedId;
+
+ /** An instance of FrameDistributor that provides the mechanism for distributing a frame to multiple readers, each operating in isolation. **/
+ private final FrameDistributor frameDistributor;
+
+ /** The original frame writer instantiated as part of job creation **/
+ private IFrameWriter writer;
+
+ /** The feed operation whose output is being distributed by the DistributeFeedFrameWriter **/
+ private final FeedRuntimeType feedRuntimeType;
+
+ /** The value of the partition 'i' if this is the i'th instance of the associated operator **/
+ private final int partition;
+
+ public DistributeFeedFrameWriter(IHyracksTaskContext ctx, FeedId feedId, IFrameWriter writer, FeedRuntimeType feedRuntimeType,
+ int partition, FrameTupleAccessor fta, IFeedManager feedManager) throws IOException {
+ this.feedId = feedId;
+ this.frameDistributor = new FrameDistributor(ctx, feedId, feedRuntimeType, partition, true,
+ feedManager.getFeedMemoryManager(), fta);
+ this.feedRuntimeType = feedRuntimeType;
+ this.partition = partition;
+ this.writer = writer;
+ }
+
+ public FeedFrameCollector subscribeFeed(FeedPolicyAccessor fpa, IFrameWriter frameWriter,
+ FeedConnectionId connectionId) throws Exception {
+ FeedFrameCollector collector = null;
+ if (!frameDistributor.isRegistered(frameWriter)) {
+ collector = new FeedFrameCollector(frameDistributor, fpa, frameWriter, connectionId);
+ frameDistributor.registerFrameCollector(collector);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Registered subscriber, new mode " + frameDistributor.getMode());
+ }
+ return collector;
+ } else {
+ throw new IllegalStateException("subscriber " + feedId + " already registered");
+ }
+ }
+
+ public void unsubscribeFeed(IFrameWriter recipientFeedFrameWriter) throws Exception {
+ boolean success = frameDistributor.deregisterFrameCollector(recipientFeedFrameWriter);
+ if (!success) {
+ throw new IllegalStateException("Invalid attempt to unregister FeedFrameWriter " + recipientFeedFrameWriter
+ + " not registered.");
+ }
+ }
+
+ public void notifyEndOfFeed() {
+ frameDistributor.notifyEndOfFeed();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ frameDistributor.close();
+ writer.close();
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer frame) throws HyracksDataException {
+ frameDistributor.nextFrame(frame);
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ writer.open();
+ }
+
+ public Map<IFrameWriter, FeedFrameCollector> getRegisteredReaders() {
+ return frameDistributor.getRegisteredReaders();
+ }
+
+ public void setWriter(IFrameWriter writer) {
+ this.writer = writer;
+ }
+
+ public Type getType() {
+ return IFeedOperatorOutputSideHandler.Type.DISTRIBUTE_FEED_OUTPUT_HANDLER;
+ }
+
+ @Override
+ public String toString() {
+ return feedId.toString() + feedRuntimeType + "[" + partition + "]";
+ }
+
+ public FrameDistributor.DistributionMode getDistributionMode() {
+ return frameDistributor.getDistributionMode();
+ }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedActivity.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedActivity.java
new file mode 100644
index 0000000..dc19555
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedActivity.java
@@ -0,0 +1,115 @@
+/*
+ * 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.common.feeds;
+
+import java.util.Map;
+
+public class FeedActivity implements Comparable<FeedActivity> {
+
+ private int activityId;
+
+ private final String dataverseName;
+ private final String datasetName;
+ private final String feedName;
+ private final Map<String, String> feedActivityDetails;
+
+ public static class FeedActivityDetails {
+ public static final String INTAKE_LOCATIONS = "intake-locations";
+ public static final String COMPUTE_LOCATIONS = "compute-locations";
+ public static final String STORAGE_LOCATIONS = "storage-locations";
+ public static final String COLLECT_LOCATIONS = "collect-locations";
+ public static final String FEED_POLICY_NAME = "feed-policy-name";
+ public static final String FEED_CONNECT_TIMESTAMP = "feed-connect-timestamp";
+
+ }
+
+ public FeedActivity(String dataverseName, String feedName, String datasetName,
+ Map<String, String> feedActivityDetails) {
+ this.dataverseName = dataverseName;
+ this.feedName = feedName;
+ this.datasetName = datasetName;
+ this.feedActivityDetails = feedActivityDetails;
+ }
+
+ public String getDataverseName() {
+ return dataverseName;
+ }
+
+ public String getDatasetName() {
+ return datasetName;
+ }
+
+ public String getFeedName() {
+ return feedName;
+ }
+
+ @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).getActivityId() != (activityId)) {
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return dataverseName + "." + feedName + " --> " + datasetName + " " + activityId;
+ }
+
+ public String getConnectTimestamp() {
+ return feedActivityDetails.get(FeedActivityDetails.FEED_CONNECT_TIMESTAMP);
+ }
+
+ public int getActivityId() {
+ return activityId;
+ }
+
+ public void setActivityId(int activityId) {
+ this.activityId = activityId;
+ }
+
+ public Map<String, String> getFeedActivityDetails() {
+ return feedActivityDetails;
+ }
+
+ @Override
+ public int compareTo(FeedActivity o) {
+ return o.getActivityId() - this.activityId;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedCollectRuntimeInputHandler.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedCollectRuntimeInputHandler.java
new file mode 100644
index 0000000..161408a
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedCollectRuntimeInputHandler.java
@@ -0,0 +1,43 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class FeedCollectRuntimeInputHandler extends FeedRuntimeInputHandler {
+
+ private final FeedFrameCache feedFrameCache;
+
+ public FeedCollectRuntimeInputHandler(IHyracksTaskContext ctx, FeedConnectionId connectionId, FeedRuntimeId runtimeId,
+ IFrameWriter coreOperator, FeedPolicyAccessor fpa, boolean bufferingEnabled,
+ FrameTupleAccessor fta, RecordDescriptor recordDesc, IFeedManager feedManager, int nPartitions)
+ throws IOException {
+ super(ctx, connectionId, runtimeId, coreOperator, fpa, bufferingEnabled, fta, recordDesc, feedManager,
+ nPartitions);
+ this.feedFrameCache = new FeedFrameCache(ctx, fta, coreOperator);
+ }
+
+ public void process(ByteBuffer frame) throws HyracksDataException {
+ feedFrameCache.sendMessage(frame);
+ super.process(frame);
+ }
+
+ public void replayFrom(int recordId) throws HyracksDataException {
+ feedFrameCache.replayRecords(recordId);
+ }
+
+ public void dropTill(int recordId) {
+ feedFrameCache.dropTillRecordId(recordId);
+ }
+
+ public void replayCached() throws HyracksDataException{
+ feedFrameCache.replayAll();
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectJobInfo.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectJobInfo.java
new file mode 100644
index 0000000..1045e6e
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectJobInfo.java
@@ -0,0 +1,75 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedJoint;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class FeedConnectJobInfo extends FeedJobInfo {
+
+ private final FeedConnectionId connectionId;
+ private final Map<String, String> feedPolicy;
+ private final IFeedJoint sourceFeedJoint;
+ private IFeedJoint computeFeedJoint;
+
+ private List<String> collectLocations;
+ private List<String> computeLocations;
+ private List<String> storageLocations;
+
+ public FeedConnectJobInfo(JobId jobId, FeedJobState state, FeedConnectionId connectionId,
+ IFeedJoint sourceFeedJoint, IFeedJoint computeFeedJoint, JobSpecification spec,
+ Map<String, String> feedPolicy) {
+ super(jobId, state, FeedJobInfo.JobType.FEED_CONNECT, spec);
+ this.connectionId = connectionId;
+ this.sourceFeedJoint = sourceFeedJoint;
+ this.computeFeedJoint = computeFeedJoint;
+ this.feedPolicy = feedPolicy;
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public List<String> getCollectLocations() {
+ return collectLocations;
+ }
+
+ public List<String> getComputeLocations() {
+ return computeLocations;
+ }
+
+ public List<String> getStorageLocations() {
+ return storageLocations;
+ }
+
+ public void setCollectLocations(List<String> collectLocations) {
+ this.collectLocations = collectLocations;
+ }
+
+ public void setComputeLocations(List<String> computeLocations) {
+ this.computeLocations = computeLocations;
+ }
+
+ public void setStorageLocations(List<String> storageLocations) {
+ this.storageLocations = storageLocations;
+ }
+
+ public IFeedJoint getSourceFeedJoint() {
+ return sourceFeedJoint;
+ }
+
+ public IFeedJoint getComputeFeedJoint() {
+ return computeFeedJoint;
+ }
+
+ public Map<String, String> getFeedPolicy() {
+ return feedPolicy;
+ }
+
+ public void setComputeFeedJoint(IFeedJoint computeFeedJoint) {
+ this.computeFeedJoint = computeFeedJoint;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectionId.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectionId.java
index def7c10..e09a4fe 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectionId.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectionId.java
@@ -17,28 +17,27 @@
import java.io.Serializable;
/**
- * A unique identifier for a data feed flowing into a dataset.
+ * A unique identifier for a feed connection. A feed connection is an instance of a data feed that is 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 FeedId feedId;
private final String datasetName;
- public FeedConnectionId(String dataverse, String feedName, String datasetName) {
- this.dataverse = dataverse;
- this.feedName = feedName;
+ public FeedConnectionId(FeedId feedId, String datasetName) {
+ this.feedId = feedId;
this.datasetName = datasetName;
}
- public String getDataverse() {
- return dataverse;
+ public FeedConnectionId(String dataverse, String feedName, String datasetName) {
+ this.feedId = new FeedId(dataverse, feedName);
+ this.datasetName = datasetName;
}
- public String getFeedName() {
- return feedName;
+ public FeedId getFeedId() {
+ return feedId;
}
public String getDatasetName() {
@@ -50,9 +49,10 @@
if (o == null || !(o instanceof FeedConnectionId)) {
return false;
}
- if (((FeedConnectionId) o).getFeedName().equals(feedName)
- && ((FeedConnectionId) o).getDataverse().equals(dataverse)
- && ((FeedConnectionId) o).getDatasetName().equals(datasetName)) {
+
+ if (this == o
+ || (((FeedConnectionId) o).getFeedId().equals(feedId) && ((FeedConnectionId) o).getDatasetName()
+ .equals(datasetName))) {
return true;
}
return false;
@@ -65,6 +65,6 @@
@Override
public String toString() {
- return dataverse + "." + feedName + "-->" + datasetName;
+ return feedId.toString() + "-->" + datasetName;
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectionRequest.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectionRequest.java
new file mode 100644
index 0000000..2343b40
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConnectionRequest.java
@@ -0,0 +1,122 @@
+/*
+ * 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.common.feeds;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang3.StringUtils;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleListener.ConnectionLocation;
+
+/**
+ * A request for connecting a feed to a dataset.
+ */
+public class FeedConnectionRequest {
+
+ public enum ConnectionStatus {
+ /** initial state upon creating a connection request **/
+ INITIALIZED,
+
+ /** connection establish; feed is receiving data **/
+ ACTIVE,
+
+ /** connection removed; feed is not receiving data **/
+ INACTIVE,
+
+ /** connection request failed **/
+ FAILED
+ }
+
+ /** Feed joint on the feed pipeline that serves as the source for this subscription **/
+ private final FeedJointKey feedJointKey;
+
+ /** Location in the source feed pipeline from where feed tuples are received. **/
+ private final ConnectionLocation connectionLocation;
+
+ /** List of functions that need to be applied in sequence after the data hand-off at the source feedPointKey. **/
+ private final List<String> functionsToApply;
+
+ /** Status associated with the subscription. */
+ private ConnectionStatus connectionStatus;
+
+ /** Name of the policy that governs feed ingestion **/
+ private final String policy;
+
+ /** Policy associated with a feed connection **/
+ private final Map<String, String> policyParameters;
+
+ /** Target dataset associated with the connection request **/
+ private final String targetDataset;
+
+ private final FeedId receivingFeedId;
+
+
+ public FeedConnectionRequest(FeedJointKey feedPointKey, ConnectionLocation connectionLocation,
+ List<String> functionsToApply, String targetDataset, String policy, Map<String, String> policyParameters,
+ FeedId receivingFeedId) {
+ this.feedJointKey = feedPointKey;
+ this.connectionLocation = connectionLocation;
+ this.functionsToApply = functionsToApply;
+ this.targetDataset = targetDataset;
+ this.policy = policy;
+ this.policyParameters = policyParameters;
+ this.receivingFeedId = receivingFeedId;
+ this.connectionStatus = ConnectionStatus.INITIALIZED;
+ }
+
+ public FeedJointKey getFeedJointKey() {
+ return feedJointKey;
+ }
+
+ public ConnectionStatus getConnectionStatus() {
+ return connectionStatus;
+ }
+
+ public void setSubscriptionStatus(ConnectionStatus connectionStatus) {
+ this.connectionStatus = connectionStatus;
+ }
+
+ public String getPolicy() {
+ return policy;
+ }
+
+ public String getTargetDataset() {
+ return targetDataset;
+ }
+
+ public ConnectionLocation getSubscriptionLocation() {
+ return connectionLocation;
+ }
+
+ public FeedId getReceivingFeedId() {
+ return receivingFeedId;
+ }
+
+ public Map<String, String> getPolicyParameters() {
+ return policyParameters;
+ }
+
+ public List<String> getFunctionsToApply() {
+ return functionsToApply;
+ }
+
+ @Override
+ public String toString() {
+ return "Feed Connection Request " + feedJointKey + " [" + connectionLocation + "]" + " Apply ("
+ + StringUtils.join(functionsToApply, ",") + ")";
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConstants.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConstants.java
new file mode 100644
index 0000000..ae29908
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedConstants.java
@@ -0,0 +1,53 @@
+package edu.uci.ics.asterix.common.feeds;
+
+public class FeedConstants {
+
+ public static final class StatisticsConstants {
+ public static final String INTAKE_TUPLEID = "intake-tupleid";
+ public static final String INTAKE_PARTITION = "intake-partition";
+ public static final String INTAKE_TIMESTAMP = "intake-timestamp";
+ public static final String COMPUTE_TIMESTAMP = "compute-timestamp";
+ public static final String STORE_TIMESTAMP = "store-timestamp";
+
+ }
+
+ public static final class MessageConstants {
+ public static final String MESSAGE_TYPE = "message-type";
+ public static final String NODE_ID = "nodeId";
+ public static final String DATAVERSE = "dataverse";
+ public static final String FEED = "feed";
+ public static final String DATASET = "dataset";
+ public static final String AQL = "aql";
+ public static final String RUNTIME_TYPE = "runtime-type";
+ public static final String PARTITION = "partition";
+ public static final String INTAKE_PARTITION = "intake-partition";
+ public static final String INFLOW_RATE = "inflow-rate";
+ public static final String OUTFLOW_RATE = "outflow-rate";
+ public static final String MODE = "mode";
+ public static final String CURRENT_CARDINALITY = "current-cardinality";
+ public static final String REDUCED_CARDINALITY = "reduced-cardinality";
+ public static final String VALUE_TYPE = "value-type";
+ public static final String VALUE = "value";
+ public static final String CPU_LOAD = "cpu-load";
+ public static final String N_RUNTIMES = "n_runtimes";
+ public static final String HEAP_USAGE = "heap_usage";
+ public static final String OPERAND_ID = "operand-id";
+ public static final String COMPUTE_PARTITION_RETAIN_LIMIT = "compute-partition-retain-limit";
+ public static final String LAST_PERSISTED_TUPLE_INTAKE_TIMESTAMP = "last-persisted-tuple-intake_timestamp";
+ public static final String PERSISTENCE_DELAY_WITHIN_LIMIT = "persistence-delay-within-limit";
+ public static final String AVERAGE_PERSISTENCE_DELAY = "average-persistence-delay";
+ public static final String COMMIT_ACKS = "commit-acks";
+ public static final String MAX_WINDOW_ACKED = "max-window-acked";
+ public static final String BASE = "base";
+ public static final String NOT_APPLICABLE = "N/A";
+
+ }
+
+ public static final class NamingConstants {
+ public static final String LIBRARY_NAME_SEPARATOR = "#";
+ }
+
+ public static class JobConstants {
+ public static final int DEFAULT_FRAME_SIZE = 8192;
+ }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedExceptionHandler.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedExceptionHandler.java
new file mode 100644
index 0000000..cd9dca4
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedExceptionHandler.java
@@ -0,0 +1,104 @@
+/*
+ * 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.common.feeds;
+
+import java.io.DataInputStream;
+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.common.exceptions.FrameDataException;
+import edu.uci.ics.asterix.common.feeds.api.IExceptionHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+public class FeedExceptionHandler implements IExceptionHandler {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedExceptionHandler.class.getName());
+
+ private final IHyracksTaskContext ctx;
+ private final FrameTupleAccessor fta;
+ private final RecordDescriptor recordDesc;
+ private final IFeedManager feedManager;
+ private final FeedConnectionId connectionId;
+
+ public FeedExceptionHandler(IHyracksTaskContext ctx, FrameTupleAccessor fta, RecordDescriptor recordDesc,
+ IFeedManager feedManager, FeedConnectionId connectionId) {
+ this.ctx = ctx;
+ this.fta = fta;
+ this.recordDesc = recordDesc;
+ this.feedManager = feedManager;
+ this.connectionId = connectionId;
+ }
+
+ public ByteBuffer handleException(Exception e, ByteBuffer frame) {
+ try {
+ if (e instanceof FrameDataException) {
+ fta.reset(frame);
+ FrameDataException fde = (FrameDataException) e;
+ int tupleIndex = fde.getTupleIndex();
+
+ // logging
+ try {
+ logExceptionCausingTuple(tupleIndex, e);
+ } catch (Exception ex) {
+ ex.addSuppressed(e);
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to log exception causing tuple due to..." + ex.getMessage());
+ }
+ }
+ // slicing
+ return FeedFrameUtil.getSlicedFrame(ctx, tupleIndex, fta);
+ } else {
+ return null;
+ }
+ } catch (Exception exception) {
+ exception.printStackTrace();
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to handle exception " + exception.getMessage());
+ }
+ return null;
+ }
+ }
+
+ private void logExceptionCausingTuple(int tupleIndex, Exception e) throws HyracksDataException, AsterixException {
+
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+ DataInputStream di = new DataInputStream(bbis);
+
+ int start = fta.getTupleStartOffset(tupleIndex) + fta.getFieldSlotsLength();
+ bbis.setByteBuffer(fta.getBuffer(), start);
+
+ Object[] record = new Object[recordDesc.getFieldCount()];
+
+ for (int i = 0; i < record.length; ++i) {
+ Object instance = recordDesc.getFields()[i].deserialize(di);
+ if (i == 0) {
+ String tuple = String.valueOf(instance);
+ feedManager.getFeedMetadataManager().logTuple(connectionId, tuple, e.getMessage(), feedManager);
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning(", " + String.valueOf(instance));
+ }
+ }
+ }
+
+ }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameCache.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameCache.java
new file mode 100644
index 0000000..310e7c0
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameCache.java
@@ -0,0 +1,167 @@
+/*
+ * 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.common.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import edu.uci.ics.asterix.common.feeds.FeedConstants.StatisticsConstants;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+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.common.comm.io.FrameTupleAppender;
+
+/**
+ * Allows caching of feed frames. This class is used in providing upstream backup.
+ * The tuples at the intake layer are held in this cache until these are acked by
+ * the storage layer post their persistence. On receiving an ack, appropriate tuples
+ * (recordsId < ackedRecordId) are dropped from the cache.
+ */
+public class FeedFrameCache extends MessageReceiver<ByteBuffer> {
+
+ /**
+ * Value represents a cache feed frame
+ * Key represents the largest record Id in the frame.
+ * At the intake side, the largest record id corresponds to the last record in the frame
+ **/
+ private final Map<Integer, ByteBuffer> orderedCache;
+ private final FrameTupleAccessor tupleAccessor;
+ private final IFrameWriter frameWriter;
+ private final IHyracksTaskContext ctx;
+
+ public FeedFrameCache(IHyracksTaskContext ctx, FrameTupleAccessor tupleAccessor, IFrameWriter frameWriter) {
+ this.tupleAccessor = tupleAccessor;
+ this.frameWriter = frameWriter;
+ /** A LinkedHashMap ensures entries are retrieved in order of their insertion **/
+ this.orderedCache = new LinkedHashMap<Integer, ByteBuffer>();
+ this.ctx = ctx;
+ }
+
+ @Override
+ public void processMessage(ByteBuffer frame) throws Exception {
+ int lastRecordId = getLastRecordId(frame);
+ ByteBuffer clone = cloneFrame(frame);
+ orderedCache.put(lastRecordId, clone);
+ }
+
+ public void dropTillRecordId(int recordId) {
+ List<Integer> dropRecordIds = new ArrayList<Integer>();
+ for (Entry<Integer, ByteBuffer> entry : orderedCache.entrySet()) {
+ int recId = entry.getKey();
+ if (recId <= recordId) {
+ dropRecordIds.add(recId);
+ } else {
+ break;
+ }
+ }
+ for (Integer r : dropRecordIds) {
+ orderedCache.remove(r);
+ }
+ }
+
+ public void replayRecords(int startingRecordId) throws HyracksDataException {
+ boolean replayPositionReached = false;
+ for (Entry<Integer, ByteBuffer> entry : orderedCache.entrySet()) {
+ // the key increases monotonically
+ int maxRecordIdInFrame = entry.getKey();
+ if (!replayPositionReached) {
+ if (startingRecordId < maxRecordIdInFrame) {
+ replayFrame(startingRecordId, entry.getValue());
+ break;
+ } else {
+ continue;
+ }
+ }
+ }
+ }
+
+ /**
+ * Replay the frame from the tuple (inclusive) with recordId as specified.
+ *
+ * @param recordId
+ * @param frame
+ * @throws HyracksDataException
+ */
+ private void replayFrame(int recordId, ByteBuffer frame) throws HyracksDataException {
+ tupleAccessor.reset(frame);
+ int nTuples = tupleAccessor.getTupleCount();
+ for (int i = 0; i < nTuples; i++) {
+ int rid = getRecordIdAtTupleIndex(i, frame);
+ if (rid == recordId) {
+ ByteBuffer slicedFrame = splitFrame(i, frame);
+ replayFrame(slicedFrame);
+ break;
+ }
+ }
+ }
+
+ private ByteBuffer splitFrame(int beginTupleIndex, ByteBuffer frame) throws HyracksDataException {
+ IFrame slicedFrame = new VSizeFrame(ctx);
+ FrameTupleAppender appender = new FrameTupleAppender();
+ appender.reset(slicedFrame, true);
+ int totalTuples = tupleAccessor.getTupleCount();
+ for (int ti = beginTupleIndex; ti < totalTuples; ti++) {
+ appender.append(tupleAccessor, ti);
+ }
+ return slicedFrame.getBuffer();
+ }
+
+ /**
+ * Replay the frame
+ *
+ * @param frame
+ * @throws HyracksDataException
+ */
+ private void replayFrame(ByteBuffer frame) throws HyracksDataException {
+ frameWriter.nextFrame(frame);
+ }
+
+ private int getLastRecordId(ByteBuffer frame) {
+ tupleAccessor.reset(frame);
+ int nTuples = tupleAccessor.getTupleCount();
+ return getRecordIdAtTupleIndex(nTuples - 1, frame);
+ }
+
+ private int getRecordIdAtTupleIndex(int tupleIndex, ByteBuffer frame) {
+ tupleAccessor.reset(frame);
+ int recordStart = tupleAccessor.getTupleStartOffset(tupleIndex) + tupleAccessor.getFieldSlotsLength();
+ int openPartOffset = frame.getInt(recordStart + 6);
+ int numOpenFields = frame.getInt(recordStart + openPartOffset);
+ int recordIdOffset = frame.getInt(recordStart + openPartOffset + 4 + numOpenFields * 8
+ + StatisticsConstants.INTAKE_TUPLEID.length() + 2 + 1);
+ int lastRecordId = frame.getInt(recordStart + recordIdOffset);
+ return lastRecordId;
+ }
+
+ private ByteBuffer cloneFrame(ByteBuffer frame) {
+ ByteBuffer clone = ByteBuffer.allocate(frame.capacity());
+ System.arraycopy(frame.array(), 0, clone.array(), 0, frame.limit());
+ return clone;
+ }
+
+ public void replayAll() throws HyracksDataException {
+ for (Entry<Integer, ByteBuffer> entry : orderedCache.entrySet()) {
+ ByteBuffer frame = entry.getValue();
+ frameWriter.nextFrame(frame);
+ }
+ }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameCollector.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameCollector.java
new file mode 100644
index 0000000..e8e424c
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameCollector.java
@@ -0,0 +1,154 @@
+/*
+ * 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
+ * Copyright 2009-2013 by The Regents of the University of California
+ * 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.common.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.api.IMessageReceiver;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class FeedFrameCollector extends MessageReceiver<DataBucket> implements IMessageReceiver<DataBucket> {
+
+ private final FeedConnectionId connectionId;
+ private final FrameDistributor frameDistributor;
+ private FeedPolicyAccessor fpa;
+ private IFrameWriter frameWriter;
+ private State state;
+
+ public enum State {
+ ACTIVE,
+ FINISHED,
+ TRANSITION,
+ HANDOVER
+ }
+
+ public FeedFrameCollector(FrameDistributor frameDistributor, FeedPolicyAccessor feedPolicyAccessor,
+ IFrameWriter frameWriter, FeedConnectionId connectionId) {
+ super();
+ this.frameDistributor = frameDistributor;
+ this.fpa = feedPolicyAccessor;
+ this.connectionId = connectionId;
+ this.frameWriter = frameWriter;
+ this.state = State.ACTIVE;
+ }
+
+ @Override
+ public void processMessage(DataBucket bucket) throws Exception {
+ try {
+ ByteBuffer frame = bucket.getContent();
+ switch (bucket.getContentType()) {
+ case DATA:
+ frameWriter.nextFrame(frame);
+ break;
+ case EOD:
+ closeCollector();
+ break;
+ case EOSD:
+ throw new AsterixException("Received data bucket with content of type " + bucket.getContentType());
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to process data bucket " + bucket + ", encountered exception " + e.getMessage());
+ }
+ } finally {
+ bucket.doneReading();
+ }
+ }
+
+ public void closeCollector() {
+ if (state.equals(State.TRANSITION)) {
+ super.close(true);
+ setState(State.ACTIVE);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(this + " is now " + State.ACTIVE + " mode, processing frames synchronously");
+ }
+ } else {
+ flushPendingMessages();
+ setState(State.FINISHED);
+ synchronized (frameDistributor.getRegisteredCollectors()) {
+ frameDistributor.getRegisteredCollectors().notifyAll();
+ }
+ disconnect();
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Closed collector " + this);
+ }
+ }
+
+ public synchronized void disconnect() {
+ setState(State.FINISHED);
+ }
+
+ public synchronized void nextFrame(ByteBuffer frame) throws HyracksDataException {
+ frameWriter.nextFrame(frame);
+ }
+
+ public FeedPolicyAccessor getFeedPolicyAccessor() {
+ return fpa;
+ }
+
+ public synchronized State getState() {
+ return state;
+ }
+
+ public synchronized void setState(State state) {
+ this.state = state;
+ switch (state) {
+ case FINISHED:
+ case HANDOVER:
+ notifyAll();
+ break;
+ default:
+ break;
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Frame Collector " + this.frameDistributor.getFeedRuntimeType() + " switched to " + state);
+ }
+ }
+
+ public IFrameWriter getFrameWriter() {
+ return frameWriter;
+ }
+
+ public void setFrameWriter(IFrameWriter frameWriter) {
+ this.frameWriter = frameWriter;
+ }
+
+ @Override
+ public String toString() {
+ return "FrameCollector " + connectionId + "," + state + "]";
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o instanceof FeedFrameCollector) {
+ return connectionId.equals(((FeedFrameCollector) o).connectionId);
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return connectionId.toString().hashCode();
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameDiscarder.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameDiscarder.java
new file mode 100644
index 0000000..437ed95
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameDiscarder.java
@@ -0,0 +1,49 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public class FeedFrameDiscarder {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedFrameSpiller.class.getName());
+
+ private final IHyracksTaskContext ctx;
+ private final FeedRuntimeInputHandler inputHandler;
+ private final FeedConnectionId connectionId;
+ private final FeedRuntimeId runtimeId;
+ private final FeedPolicyAccessor policyAccessor;
+ private final float maxFractionDiscard;
+ private int nDiscarded;
+
+ public FeedFrameDiscarder(IHyracksTaskContext ctx, FeedConnectionId connectionId, FeedRuntimeId runtimeId,
+ FeedPolicyAccessor policyAccessor, FeedRuntimeInputHandler inputHandler) throws IOException {
+ this.ctx = ctx;
+ this.connectionId = connectionId;
+ this.runtimeId = runtimeId;
+ this.policyAccessor = policyAccessor;
+ this.inputHandler = inputHandler;
+ this.maxFractionDiscard = policyAccessor.getMaxFractionDiscard();
+ }
+
+ public boolean processMessage(ByteBuffer message) {
+ if (policyAccessor.getMaxFractionDiscard() != 0) {
+ long nProcessed = inputHandler.getProcessed();
+ long discardLimit = (long) (nProcessed * maxFractionDiscard);
+ if (nDiscarded >= discardLimit) {
+ return false;
+ }
+ nDiscarded++;
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Discarded frame by " + connectionId + " (" + runtimeId + ")" + " count so far ("
+ + nDiscarded + ") Limit [" + discardLimit + "]");
+ }
+ return true;
+ }
+ return false;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameHandlers.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameHandlers.java
new file mode 100644
index 0000000..5c41fd4
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameHandlers.java
@@ -0,0 +1,302 @@
+/*
+ * 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.common.feeds;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedFrameHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class FeedFrameHandlers {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedFrameHandlers.class.getName());
+
+ public enum RoutingMode {
+ IN_MEMORY_ROUTE,
+ SPILL_TO_DISK,
+ DISCARD
+ }
+
+ public static IFeedFrameHandler getFeedFrameHandler(FrameDistributor distributor, FeedId feedId,
+ RoutingMode routingMode, FeedRuntimeType runtimeType, int partition, int frameSize) throws IOException {
+ IFeedFrameHandler handler = null;
+ switch (routingMode) {
+ case IN_MEMORY_ROUTE:
+ handler = new InMemoryRouter(distributor.getRegisteredReaders().values(), runtimeType, partition);
+ break;
+ case SPILL_TO_DISK:
+ handler = new DiskSpiller(distributor, feedId, runtimeType, partition, frameSize);
+ break;
+ case DISCARD:
+ handler = new DiscardRouter(distributor, feedId, runtimeType, partition);
+ break;
+ default:
+ throw new IllegalArgumentException("Invalid routing mode" + routingMode);
+ }
+ return handler;
+ }
+
+ public static class DiscardRouter implements IFeedFrameHandler {
+
+ private final FeedId feedId;
+ private int nDiscarded;
+ private final FeedRuntimeType runtimeType;
+ private final int partition;
+ private final FrameDistributor distributor;
+
+ public DiscardRouter(FrameDistributor distributor, FeedId feedId, FeedRuntimeType runtimeType, int partition)
+ throws HyracksDataException {
+ this.distributor = distributor;
+ this.feedId = feedId;
+ this.nDiscarded = 0;
+ this.runtimeType = runtimeType;
+ this.partition = partition;
+ }
+
+ @Override
+ public void handleFrame(ByteBuffer frame) throws HyracksDataException {
+ FrameTupleAccessor fta = distributor.getFta();
+ fta.reset(frame);
+ int nTuples = fta.getTupleCount();
+ nDiscarded += nTuples;
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Discarded additional [" + runtimeType + "]" + "(" + partition + ")" + " " + nTuples);
+ }
+ }
+
+ @Override
+ public void handleDataBucket(DataBucket bucket) {
+ nDiscarded++;
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Discard Count" + nDiscarded);
+ }
+ }
+
+ @Override
+ public void close() {
+ // do nothing, no resource to relinquish
+ }
+
+ @Override
+ public Iterator<ByteBuffer> replayData() throws HyracksDataException {
+ throw new IllegalStateException("Invalid operation");
+ }
+
+ @Override
+ public String toString() {
+ return "DiscardRouter" + "[" + feedId + "]" + "(" + nDiscarded + ")";
+ }
+
+ @Override
+ public String getSummary() {
+ return new String("Number of discarded frames (since last reset)" + " feedId " + "[" + feedId + "]" + "("
+ + nDiscarded + ")");
+ }
+
+ }
+
+ public static class InMemoryRouter implements IFeedFrameHandler {
+
+ private final Collection<FeedFrameCollector> frameCollectors;
+
+ public InMemoryRouter(Collection<FeedFrameCollector> frameCollectors, FeedRuntimeType runtimeType, int partition) {
+ this.frameCollectors = frameCollectors;
+ }
+
+ @Override
+ public void handleFrame(ByteBuffer frame) throws HyracksDataException {
+ throw new IllegalStateException("Operation not supported");
+ }
+
+ @Override
+ public void handleDataBucket(DataBucket bucket) {
+ for (FeedFrameCollector collector : frameCollectors) {
+ collector.sendMessage(bucket); // asynchronous call
+ }
+ }
+
+ @Override
+ public void close() {
+ // do nothing
+ }
+
+ @Override
+ public Iterator<ByteBuffer> replayData() throws HyracksDataException {
+ throw new IllegalStateException("Operation not supported");
+ }
+
+ @Override
+ public String getSummary() {
+ return "InMemoryRouter Summary";
+ }
+ }
+
+ public static class DiskSpiller implements IFeedFrameHandler {
+
+ private final FeedId feedId;
+ private FrameSpiller<ByteBuffer> receiver;
+ private Iterator<ByteBuffer> iterator;
+
+ public DiskSpiller(FrameDistributor distributor, FeedId feedId, FeedRuntimeType runtimeType, int partition,
+ int frameSize) throws IOException {
+ this.feedId = feedId;
+ receiver = new FrameSpiller<ByteBuffer>(distributor, feedId, frameSize);
+ }
+
+ @Override
+ public void handleFrame(ByteBuffer frame) throws HyracksDataException {
+ receiver.sendMessage(frame);
+ }
+
+ private static class FrameSpiller<T> extends MessageReceiver<ByteBuffer> {
+
+ private final int frameSize;
+ private final FeedId feedId;
+ private BufferedOutputStream bos;
+ private final ByteBuffer reusableLengthBuffer;
+ private final ByteBuffer reusableDataBuffer;
+ private long offset;
+ private File file;
+ private final FrameDistributor frameDistributor;
+ private boolean fileCreated = false;
+
+ public FrameSpiller(FrameDistributor distributor, FeedId feedId, int frameSize) throws IOException {
+ this.feedId = feedId;
+ this.frameSize = frameSize;
+ this.frameDistributor = distributor;
+ reusableLengthBuffer = ByteBuffer.allocate(4);
+ reusableDataBuffer = ByteBuffer.allocate(frameSize);
+ this.offset = 0;
+ }
+
+ @Override
+ public void processMessage(ByteBuffer message) throws Exception {
+ if (!fileCreated) {
+ createFile();
+ fileCreated = true;
+ }
+ reusableLengthBuffer.flip();
+ reusableLengthBuffer.putInt(message.array().length);
+ bos.write(reusableLengthBuffer.array());
+ bos.write(message.array());
+ }
+
+ private void createFile() throws IOException {
+ Date date = new Date();
+ String dateSuffix = date.toString().replace(' ', '_');
+ String fileName = feedId.toString() + "_" + frameDistributor.getFeedRuntimeType() + "_"
+ + frameDistributor.getPartition() + "_" + dateSuffix;
+
+ file = new File(fileName);
+ if (!file.exists()) {
+ boolean success = file.createNewFile();
+ if (!success) {
+ throw new IOException("Unable to create spill file for feed " + feedId);
+ }
+ }
+ bos = new BufferedOutputStream(new FileOutputStream(file));
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Created Spill File for feed " + feedId);
+ }
+ }
+
+ @SuppressWarnings("resource")
+ public Iterator<ByteBuffer> replayData() throws Exception {
+ final BufferedInputStream bis = new BufferedInputStream(new FileInputStream(file));
+ bis.skip(offset);
+ return new Iterator<ByteBuffer>() {
+
+ @Override
+ public boolean hasNext() {
+ boolean more = false;
+ try {
+ more = bis.available() > 0;
+ if (!more) {
+ bis.close();
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+
+ return more;
+ }
+
+ @Override
+ public ByteBuffer next() {
+ reusableLengthBuffer.flip();
+ try {
+ bis.read(reusableLengthBuffer.array());
+ reusableLengthBuffer.flip();
+ int frameSize = reusableLengthBuffer.getInt();
+ reusableDataBuffer.flip();
+ bis.read(reusableDataBuffer.array(), 0, frameSize);
+ offset += 4 + frameSize;
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ return reusableDataBuffer;
+ }
+
+ @Override
+ public void remove() {
+ }
+
+ };
+ }
+
+ }
+
+ @Override
+ public void handleDataBucket(DataBucket bucket) {
+ throw new IllegalStateException("Operation not supported");
+ }
+
+ @Override
+ public void close() {
+ receiver.close(true);
+ }
+
+ @Override
+ public Iterator<ByteBuffer> replayData() throws HyracksDataException {
+ try {
+ iterator = receiver.replayData();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ return iterator;
+ }
+
+ //TODO: Form a summary that includes stats related to what has been spilled to disk
+ @Override
+ public String getSummary() {
+ return "Disk Spiller Summary";
+ }
+
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameSpiller.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameSpiller.java
new file mode 100644
index 0000000..db8145f
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameSpiller.java
@@ -0,0 +1,172 @@
+/*
+ * 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.common.feeds;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public class FeedFrameSpiller {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedFrameSpiller.class.getName());
+
+ private final IHyracksTaskContext ctx;
+ private final FeedConnectionId connectionId;
+ private final FeedRuntimeId runtimeId;
+ private final FeedPolicyAccessor policyAccessor;
+ private BufferedOutputStream bos;
+ private File file;
+ private boolean fileCreated = false;
+ private long bytesWritten = 0;
+ private int spilledFrameCount = 0;
+
+ public FeedFrameSpiller(IHyracksTaskContext ctx, FeedConnectionId connectionId, FeedRuntimeId runtimeId,
+ FeedPolicyAccessor policyAccessor) throws IOException {
+ this.ctx = ctx;
+ this.connectionId = connectionId;
+ this.runtimeId = runtimeId;
+ this.policyAccessor = policyAccessor;
+ }
+
+ public boolean processMessage(ByteBuffer message) throws Exception {
+ if (!fileCreated) {
+ createFile();
+ fileCreated = true;
+ }
+ long maxAllowed = policyAccessor.getMaxSpillOnDisk();
+ if (maxAllowed != FeedPolicyAccessor.NO_LIMIT && bytesWritten + message.array().length > maxAllowed) {
+ return false;
+ } else {
+ bos.write(message.array());
+ bytesWritten += message.array().length;
+ spilledFrameCount++;
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Spilled frame by " + runtimeId + " spill count " + spilledFrameCount);
+ }
+ return true;
+ }
+ }
+
+ private void createFile() throws IOException {
+ Date date = new Date();
+ String dateSuffix = date.toString().replace(' ', '_');
+ String fileName = connectionId.getFeedId() + "_" + connectionId.getDatasetName() + "_"
+ + runtimeId.getFeedRuntimeType() + "_" + runtimeId.getPartition() + "_" + dateSuffix;
+
+ file = new File(fileName);
+ if (!file.exists()) {
+ boolean success = file.createNewFile();
+ if (!success) {
+ throw new IOException("Unable to create spill file " + fileName + " for feed " + runtimeId);
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Created spill file " + file.getAbsolutePath());
+ }
+ }
+ }
+ bos = new BufferedOutputStream(new FileOutputStream(file));
+
+ }
+
+ public Iterator<ByteBuffer> replayData() throws Exception {
+ bos.flush();
+ return new FrameIterator(ctx, file.getName());
+ }
+
+ private static class FrameIterator implements Iterator<ByteBuffer> {
+
+ private final BufferedInputStream bis;
+ private final IHyracksTaskContext ctx;
+ private int readFrameCount = 0;
+
+ public FrameIterator(IHyracksTaskContext ctx, String filename) throws FileNotFoundException {
+ bis = new BufferedInputStream(new FileInputStream(new File(filename)));
+ this.ctx = ctx;
+ }
+
+ @Override
+ public boolean hasNext() {
+ boolean more = false;
+ try {
+ more = bis.available() > 0;
+ if (!more) {
+ bis.close();
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+
+ return more;
+ }
+
+ @Override
+ public ByteBuffer next() {
+ IFrame frame = null;
+ try {
+ frame = new VSizeFrame(ctx);
+ Arrays.fill(frame.getBuffer().array(), (byte) 0);
+ bis.read(frame.getBuffer().array(), 0, frame.getFrameSize());
+ readFrameCount++;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Read spill frome " + readFrameCount);
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ return frame.getBuffer();
+ }
+
+ @Override
+ public void remove() {
+ }
+
+ }
+
+ public void reset() {
+ bytesWritten = 0;
+ // file.delete();
+ fileCreated = false;
+ bos = null;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Resetted the FrameSpiller!");
+ }
+ }
+
+ public void close() {
+ if (bos != null) {
+ try {
+ bos.flush();
+ bos.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameTupleAccessor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameTupleAccessor.java
new file mode 100644
index 0000000..9567307
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameTupleAccessor.java
@@ -0,0 +1,92 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.common.feeds.FeedConstants.StatisticsConstants;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class FeedFrameTupleAccessor implements IFrameTupleAccessor {
+
+ private final FrameTupleAccessor frameAccessor;
+ private final int numOpenFields;
+
+ public FeedFrameTupleAccessor(FrameTupleAccessor frameAccessor) {
+ this.frameAccessor = frameAccessor;
+ int firstRecordStart = frameAccessor.getTupleStartOffset(0) + frameAccessor.getFieldSlotsLength();
+ int openPartOffsetOrig = frameAccessor.getBuffer().getInt(firstRecordStart + 6);
+ numOpenFields = frameAccessor.getBuffer().getInt(firstRecordStart + openPartOffsetOrig);
+ }
+
+ public int getFeedIntakePartition(int tupleIndex) {
+ ByteBuffer buffer = frameAccessor.getBuffer();
+ int recordStart = frameAccessor.getTupleStartOffset(tupleIndex) + frameAccessor.getFieldSlotsLength();
+ int openPartOffsetOrig = buffer.getInt(recordStart + 6);
+ int partitionOffset = openPartOffsetOrig + 4 + 8 * numOpenFields
+ + StatisticsConstants.INTAKE_PARTITION.length() + 2 + 1;
+ return buffer.getInt(recordStart + partitionOffset);
+ }
+
+
+
+ @Override
+ public int getFieldCount() {
+ return frameAccessor.getFieldCount();
+ }
+
+ @Override
+ public int getFieldSlotsLength() {
+ return frameAccessor.getFieldSlotsLength();
+ }
+
+ @Override
+ public int getFieldEndOffset(int tupleIndex, int fIdx) {
+ return frameAccessor.getFieldEndOffset(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getFieldStartOffset(int tupleIndex, int fIdx) {
+ return frameAccessor.getFieldStartOffset(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getFieldLength(int tupleIndex, int fIdx) {
+ return frameAccessor.getFieldLength(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getTupleEndOffset(int tupleIndex) {
+ return frameAccessor.getTupleEndOffset(tupleIndex);
+ }
+
+ @Override
+ public int getTupleStartOffset(int tupleIndex) {
+ return frameAccessor.getTupleStartOffset(tupleIndex);
+ }
+
+ @Override
+ public int getTupleCount() {
+ return frameAccessor.getTupleCount();
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return frameAccessor.getBuffer();
+ }
+
+ @Override
+ public void reset(ByteBuffer buffer) {
+ frameAccessor.reset(buffer);
+ }
+
+ @Override
+ public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+ return frameAccessor.getAbsoluteFieldStartOffset(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getTupleLength(int tupleIndex) {
+ return frameAccessor.getTupleLength(tupleIndex);
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameUtil.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameUtil.java
new file mode 100644
index 0000000..927d10d
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedFrameUtil.java
@@ -0,0 +1,84 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+import java.util.Random;
+
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+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.common.comm.io.FrameTupleAppender;
+
+public class FeedFrameUtil {
+
+ public static ByteBuffer getSlicedFrame(IHyracksTaskContext ctx, int tupleIndex, FrameTupleAccessor fta) throws HyracksDataException {
+ FrameTupleAppender appender = new FrameTupleAppender();
+ IFrame slicedFrame = new VSizeFrame(ctx);
+ appender.reset(slicedFrame, true);
+ int startTupleIndex = tupleIndex + 1;
+ int totalTuples = fta.getTupleCount();
+ for (int ti = startTupleIndex; ti < totalTuples; ti++) {
+ appender.append(fta, ti);
+ }
+ return slicedFrame.getBuffer();
+ }
+
+ public static ByteBuffer getSampledFrame(IHyracksTaskContext ctx, FrameTupleAccessor fta, int sampleSize) throws HyracksDataException {
+ NChooseKIterator it = new NChooseKIterator(fta.getTupleCount(), sampleSize);
+ FrameTupleAppender appender = new FrameTupleAppender();
+ IFrame sampledFrame = new VSizeFrame(ctx);
+ appender.reset(sampledFrame, true);
+ int nextTupleIndex = 0;
+ while (it.hasNext()) {
+ nextTupleIndex = it.next();
+ appender.append(fta, nextTupleIndex);
+ }
+ return sampledFrame.getBuffer();
+ }
+
+
+
+ private static class NChooseKIterator {
+
+ private final int n;
+ private final int k;
+ private final BitSet bSet;
+ private final Random random;
+
+ private int traversed = 0;
+
+ public NChooseKIterator(int n, int k) {
+ this.n = n;
+ this.k = k;
+ this.bSet = new BitSet(n);
+ bSet.set(0, n - 1);
+ this.random = new Random();
+ }
+
+ public boolean hasNext() {
+ return traversed < k;
+ }
+
+ public int next() {
+ if (hasNext()) {
+ traversed++;
+ int startOffset = random.nextInt(n);
+ int pos = -1;
+ while (pos < 0) {
+ pos = bSet.nextSetBit(startOffset);
+ if (pos < 0) {
+ startOffset = 0;
+ }
+ }
+ bSet.clear(pos);
+ return pos;
+ } else {
+ return -1;
+ }
+ }
+
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedId.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedId.java
new file mode 100644
index 0000000..1e38f70
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedId.java
@@ -0,0 +1,62 @@
+/*
+ * 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.common.feeds;
+
+import java.io.Serializable;
+
+/**
+ * A unique identifier for a data feed.
+ */
+public class FeedId implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String dataverse;
+ private final String feedName;
+
+ public FeedId(String dataverse, String feedName) {
+ this.dataverse = dataverse;
+ this.feedName = feedName;
+ }
+
+ public String getDataverse() {
+ return dataverse;
+ }
+
+ public String getFeedName() {
+ return feedName;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == null || !(o instanceof FeedId)) {
+ return false;
+ }
+ if (this == o || ((FeedId) o).getFeedName().equals(feedName) && ((FeedId) o).getDataverse().equals(dataverse)) {
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return dataverse + "." + feedName;
+ }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedIntakeInfo.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedIntakeInfo.java
new file mode 100644
index 0000000..8ed3cf1
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedIntakeInfo.java
@@ -0,0 +1,44 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedJoint;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class FeedIntakeInfo extends FeedJobInfo {
+
+ private final FeedId feedId;
+ private final IFeedJoint intakeFeedJoint;
+ private final JobSpecification spec;
+ private List<String> intakeLocation;
+
+ public FeedIntakeInfo(JobId jobId, FeedJobState state, JobType jobType, FeedId feedId, IFeedJoint intakeFeedJoint,
+ JobSpecification spec) {
+ super(jobId, state, FeedJobInfo.JobType.INTAKE, spec);
+ this.feedId = feedId;
+ this.intakeFeedJoint = intakeFeedJoint;
+ this.spec = spec;
+ }
+
+ public FeedId getFeedId() {
+ return feedId;
+ }
+
+ public IFeedJoint getIntakeFeedJoint() {
+ return intakeFeedJoint;
+ }
+
+ public JobSpecification getSpec() {
+ return spec;
+ }
+
+ public List<String> getIntakeLocation() {
+ return intakeLocation;
+ }
+
+ public void setIntakeLocation(List<String> intakeLocation) {
+ this.intakeLocation = intakeLocation;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedJobInfo.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedJobInfo.java
new file mode 100644
index 0000000..c50ca43
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedJobInfo.java
@@ -0,0 +1,68 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class FeedJobInfo {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedJobInfo.class.getName());
+
+ public enum JobType {
+ INTAKE,
+ FEED_CONNECT
+ }
+
+ public enum FeedJobState {
+ CREATED,
+ ACTIVE,
+ UNDER_RECOVERY,
+ ENDED
+ }
+
+ protected final JobId jobId;
+ protected final JobType jobType;
+ protected FeedJobState state;
+ protected JobSpecification spec;
+
+ public FeedJobInfo(JobId jobId, FeedJobState state, JobType jobType, JobSpecification spec) {
+ this.jobId = jobId;
+ this.state = state;
+ this.jobType = jobType;
+ this.spec = spec;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public FeedJobState getState() {
+ return state;
+ }
+
+ public void setState(FeedJobState state) {
+ this.state = state;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(this + " is in " + state + " state.");
+ }
+ }
+
+ public JobType getJobType() {
+ return jobType;
+ }
+
+ public JobSpecification getSpec() {
+ return spec;
+ }
+
+ public void setSpec(JobSpecification spec) {
+ this.spec = spec;
+ }
+
+ public String toString() {
+ return jobId + " [" + jobType + "]";
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedJointKey.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedJointKey.java
new file mode 100644
index 0000000..145ac32
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedJointKey.java
@@ -0,0 +1,75 @@
+/*
+ * 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.common.feeds;
+
+import java.util.List;
+
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * Represents a unique identifier for a Feed Joint. A Feed joint is a logical entity located
+ * along a feed ingestion pipeline at a point where the tuples moving as part of data flow
+ * constitute the feed. The feed joint acts as a network tap and allows the flowing data to be
+ * routed to multiple paths.
+ */
+public class FeedJointKey {
+
+ private final FeedId primaryFeedId;
+ private final List<String> appliedFunctions;
+ private final String stringRep;
+
+ public FeedJointKey(FeedId feedId, List<String> appliedFunctions) {
+ this.primaryFeedId = feedId;
+ this.appliedFunctions = appliedFunctions;
+ StringBuilder builder = new StringBuilder();
+ builder.append(feedId);
+ builder.append(":");
+ builder.append(StringUtils.join(appliedFunctions, ':'));
+ stringRep = builder.toString();
+ }
+
+ public FeedId getFeedId() {
+ return primaryFeedId;
+ }
+
+ public List<String> getAppliedFunctions() {
+ return appliedFunctions;
+ }
+
+ public String getStringRep() {
+ return stringRep;
+ }
+
+ @Override
+ public final String toString() {
+ return stringRep;
+ }
+
+ @Override
+ public int hashCode() {
+ return stringRep.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o)
+ return true;
+ if (o == null || !(o instanceof FeedJointKey)) {
+ return false;
+ }
+ return stringRep.equals(((FeedJointKey) o).stringRep);
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMemoryManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMemoryManager.java
new file mode 100644
index 0000000..88e900d
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMemoryManager.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.common.feeds;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.config.AsterixFeedProperties;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryComponent;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryComponent.Type;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryManager;
+
+public class FeedMemoryManager implements IFeedMemoryManager {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedMemoryManager.class.getName());
+ private static final int ALLOCATION_INCREMENT = 10;
+
+ private final AtomicInteger componentId = new AtomicInteger(0);
+ private final String nodeId;
+ private final int budget;
+ private final int frameSize;
+
+ private int committed;
+
+ public FeedMemoryManager(String nodeId, AsterixFeedProperties feedProperties, int frameSize) {
+ this.nodeId = nodeId;
+ this.frameSize = frameSize;
+ budget = (int) feedProperties.getMemoryComponentGlobalBudget() / frameSize;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Feed Memory budget " + budget + " frames (frame size=" + frameSize + ")");
+ }
+ }
+
+ @Override
+ public synchronized IFeedMemoryComponent getMemoryComponent(Type type) {
+ IFeedMemoryComponent memoryComponent = null;
+ boolean valid = false;
+ switch (type) {
+ case COLLECTION:
+ valid = committed + START_COLLECTION_SIZE <= budget;
+ if (valid) {
+ memoryComponent = new FrameCollection(componentId.incrementAndGet(), this, START_COLLECTION_SIZE);
+ }
+ break;
+ case POOL:
+ valid = committed + START_POOL_SIZE <= budget;
+ if (valid) {
+ memoryComponent = new DataBucketPool(componentId.incrementAndGet(), this, START_POOL_SIZE,
+ frameSize);
+ }
+ committed += START_POOL_SIZE;
+ break;
+ }
+ if (!valid) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to allocate memory component of type" + type);
+ }
+ }
+ return valid ? memoryComponent : null;
+ }
+
+ @Override
+ public synchronized boolean expandMemoryComponent(IFeedMemoryComponent memoryComponent) {
+ if (committed + ALLOCATION_INCREMENT > budget) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Memory budget " + budget + " is exhausted. Space left: " + (budget - committed)
+ + " frames.");
+ }
+ return false;
+ } else {
+ memoryComponent.expand(ALLOCATION_INCREMENT);
+ committed += ALLOCATION_INCREMENT;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Expanded memory component " + memoryComponent + " by " + ALLOCATION_INCREMENT + " " + this);
+ }
+ return true;
+ }
+ }
+
+ @Override
+ public synchronized void releaseMemoryComponent(IFeedMemoryComponent memoryComponent) {
+ int delta = memoryComponent.getTotalAllocation();
+ committed -= delta;
+ memoryComponent.reset();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Reset " + memoryComponent + " and reclaimed " + delta + " frames " + this);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "FeedMemoryManager [" + nodeId + "]" + "(" + committed + "/" + budget + ")";
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMessageService.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMessageService.java
index 3d0d8f9..320e6b9 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMessageService.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMessageService.java
@@ -1,149 +1,140 @@
+/*
+ * 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.common.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.ExecutorService;
+import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.logging.Level;
import java.util.logging.Logger;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.config.AsterixFeedProperties;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessage;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessageService;
+
/**
* Sends feed report messages on behalf of an operator instance
* to the SuperFeedManager associated with the feed.
*/
-public class FeedMessageService {
+public class FeedMessageService implements IFeedMessageService {
private static final Logger LOGGER = Logger.getLogger(FeedMessageService.class.getName());
- public static final char MessageSeparator = '|';
- private static final char EOL = (char) "\n".getBytes()[0];
-
- private final FeedConnectionId feedId;
private final LinkedBlockingQueue<String> inbox;
private final FeedMessageHandler mesgHandler;
- private final IFeedManager feedManager;
+ private final String nodeId;
+ private ExecutorService executor;
- public FeedMessageService(FeedConnectionId feedId, IFeedManager feedManager) {
- this.feedId = feedId;
- inbox = new LinkedBlockingQueue<String>();
- mesgHandler = new FeedMessageHandler(inbox, feedId, feedManager);
- this.feedManager = feedManager;
+ public FeedMessageService(AsterixFeedProperties feedProperties, String nodeId, String ccClusterIp) {
+ this.inbox = new LinkedBlockingQueue<String>();
+ this.mesgHandler = new FeedMessageHandler(inbox, ccClusterIp, feedProperties.getFeedCentralManagerPort());
+ this.nodeId = nodeId;
+ this.executor = Executors.newSingleThreadExecutor();
}
- public void start() throws UnknownHostException, IOException, Exception {
- feedManager.getFeedExecutorService(feedId).execute(mesgHandler);
+ public void start() throws Exception {
+
+ executor.execute(mesgHandler);
}
- public void stop() throws IOException {
+ public void stop() {
+ synchronized (mesgHandler.getLock()) {
+ executor.shutdownNow();
+ }
mesgHandler.stop();
}
- public void sendMessage(String message) throws IOException {
- inbox.add(message);
+ @Override
+ public void sendMessage(IFeedMessage message) {
+ try {
+ JSONObject obj = message.toJSON();
+ obj.put(FeedConstants.MessageConstants.NODE_ID, nodeId);
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, message.getMessageType().name());
+ inbox.add(obj.toString());
+ } catch (JSONException jse) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("JSON exception in parsing message " + message + " exception [" + jse.getMessage() + "]");
+ }
+ }
}
private static class FeedMessageHandler implements Runnable {
private final LinkedBlockingQueue<String> inbox;
- private final FeedConnectionId feedId;
- private Socket sfmSocket;
- private boolean process = true;
- private final IFeedManager feedManager;
+ private final String host;
+ private final int port;
+ private final Object lock;
- public FeedMessageHandler(LinkedBlockingQueue<String> inbox, FeedConnectionId feedId, IFeedManager feedManager) {
+ private Socket cfmSocket;
+
+ private static final byte[] EOL = "\n".getBytes();
+
+ public FeedMessageHandler(LinkedBlockingQueue<String> inbox, String host, int port) {
this.inbox = inbox;
- this.feedId = feedId;
- this.feedManager = feedManager;
+ this.host = host;
+ this.port = port;
+ this.lock = new Object();
}
public void run() {
try {
- sfmSocket = obtainSFMSocket();
- if (sfmSocket != null) {
- while (process) {
+ cfmSocket = new Socket(host, port);
+ if (cfmSocket != null) {
+ while (true) {
String message = inbox.take();
- sfmSocket.getOutputStream().write(message.getBytes());
+ synchronized (lock) { // lock prevents message handler from sending incomplete message midst shutdown attempt
+ cfmSocket.getOutputStream().write(message.getBytes());
+ cfmSocket.getOutputStream().write(EOL);
+ }
}
} else {
if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to start feed message service for " + feedId);
+ LOGGER.warning("Unable to start feed message service");
}
}
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Ended feed message service for " + feedId);
- }
} catch (Exception e) {
+ e.printStackTrace();
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());
- }
- }
- }
+ stop();
}
}
public void stop() {
- process = false;
- }
-
- private Socket obtainSFMSocket() throws UnknownHostException, IOException, Exception {
- Socket sfmDirServiceSocket = null;
- SuperFeedManager sfm = feedManager.getSuperFeedManager(feedId);
- try {
- FeedRuntimeManager runtimeManager = feedManager.getFeedRuntimeManager(feedId);
- sfmDirServiceSocket = runtimeManager.createClientSocket(sfm.getHost(), sfm.getPort(),
- IFeedManager.SOCKET_CONNECT_TIMEOUT);
- if (sfmDirServiceSocket == null) {
+ if (cfmSocket != null) {
+ try {
+ cfmSocket.close();
+ } catch (Exception e) {
if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to connect to " + sfm.getHost() + "[" + sfm.getPort() + "]");
+ LOGGER.warning("Exception in closing socket " + e.getMessage());
}
- } 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,
- IFeedManager.SOCKET_CONNECT_TIMEOUT);
- }
- } catch (Exception e) {
- e.printStackTrace();
- throw e;
- } finally {
- if (sfmDirServiceSocket != null) {
- sfmDirServiceSocket.close();
}
}
- return sfmSocket;
}
+
+ public Object getLock() {
+ return lock;
+ }
+
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMetricCollector.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMetricCollector.java
new file mode 100644
index 0000000..ff943e1
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedMetricCollector.java
@@ -0,0 +1,185 @@
+/*
+ * 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.common.feeds;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector;
+
+public class FeedMetricCollector implements IFeedMetricCollector {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedMetricCollector.class.getName());
+
+ private static final int UNKNOWN = -1;
+
+ private final String nodeId;
+ private final AtomicInteger globalSenderId = new AtomicInteger(1);
+ private final Map<Integer, Sender> senders = new HashMap<Integer, Sender>();
+ private final Map<Integer, Series> statHistory = new HashMap<Integer, Series>();
+ private final Map<String, Sender> sendersByName = new HashMap<String, Sender>();
+
+ public FeedMetricCollector(String nodeId) {
+ this.nodeId = nodeId;
+ }
+
+ @Override
+ public synchronized int createReportSender(FeedConnectionId connectionId, FeedRuntimeId runtimeId,
+ ValueType valueType, MetricType metricType) {
+ Sender sender = new Sender(globalSenderId.getAndIncrement(), connectionId, runtimeId, valueType, metricType);
+ senders.put(sender.senderId, sender);
+ sendersByName.put(sender.getDisplayName(), sender);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Sender id " + sender.getSenderId() + " created for " + sender);
+ }
+ return sender.senderId;
+ }
+
+ @Override
+ public void removeReportSender(int senderId) {
+ Sender sender = senders.get(senderId);
+ if (sender != null) {
+ statHistory.remove(senderId);
+ senders.remove(senderId);
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to remove sender Id");
+ }
+ throw new IllegalStateException("Unable to remove sender Id " + senderId + " senders " + senders);
+ }
+ }
+
+ @Override
+ public boolean sendReport(int senderId, int value) {
+ Sender sender = senders.get(senderId);
+ if (sender != null) {
+ Series series = statHistory.get(sender.senderId);
+ if (series == null) {
+ switch (sender.mType) {
+ case AVG:
+ series = new SeriesAvg();
+ break;
+ case RATE:
+ series = new SeriesRate();
+ break;
+ }
+ statHistory.put(sender.senderId, series);
+ }
+ series.addValue(value);
+ return true;
+ }
+ throw new IllegalStateException("Unable to send report sender Id " + senderId + " senders " + senders);
+ }
+
+ @Override
+ public void resetReportSender(int senderId) {
+ Sender sender = senders.get(senderId);
+ if (sender != null) {
+ Series series = statHistory.get(sender.senderId);
+ if (series != null) {
+ series.reset();
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Sender with id " + senderId + " not found. Unable to reset!");
+ }
+ throw new IllegalStateException("Unable to reset sender Id " + senderId + " senders " + senders);
+ }
+ }
+
+ private static class Sender {
+
+ private final int senderId;
+ private final MetricType mType;
+ private final String displayName;
+
+ public Sender(int senderId, FeedConnectionId connectionId, FeedRuntimeId runtimeId, ValueType valueType,
+ MetricType mType) {
+ this.senderId = senderId;
+ this.mType = mType;
+ this.displayName = createDisplayName(connectionId, runtimeId, valueType);
+ }
+
+ @Override
+ public String toString() {
+ return displayName + "[" + senderId + "]" + "(" + mType + ")";
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof Sender)) {
+ return false;
+ }
+ return ((Sender) o).senderId == senderId;
+ }
+
+ @Override
+ public int hashCode() {
+ return senderId;
+ }
+
+ public static String createDisplayName(FeedConnectionId connectionId, FeedRuntimeId runtimeId,
+ ValueType valueType) {
+ return connectionId + " (" + runtimeId.getFeedRuntimeType() + " )" + "[" + runtimeId.getPartition() + "]"
+ + "{" + valueType + "}";
+ }
+
+ public String getDisplayName() {
+ return displayName;
+ }
+
+ public int getSenderId() {
+ return senderId;
+ }
+ }
+
+ @Override
+ public int getMetric(int senderId) {
+ Sender sender = senders.get(senderId);
+ return getMetric(sender);
+ }
+
+ @Override
+ public int getMetric(FeedConnectionId connectionId, FeedRuntimeId runtimeId, ValueType valueType) {
+ String displayName = Sender.createDisplayName(connectionId, runtimeId, valueType);
+ Sender sender = sendersByName.get(displayName);
+ return getMetric(sender);
+ }
+
+ private int getMetric(Sender sender) {
+ if (sender == null || statHistory.get(sender.getSenderId()) == null) {
+ return UNKNOWN;
+ }
+
+ float result = -1;
+ Series series = statHistory.get(sender.getSenderId());
+ switch (sender.mType) {
+ case AVG:
+ result = ((SeriesAvg) series).getAvg();
+ break;
+ case RATE:
+ result = ((SeriesRate) series).getRate();
+ break;
+ }
+ return (int) result;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedPolicyAccessor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedPolicyAccessor.java
new file mode 100644
index 0000000..6e6ce75
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedPolicyAccessor.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.common.feeds;
+
+import java.io.Serializable;
+import java.util.Map;
+
+/**
+ * A utility class to access the configuration parameters of a feed ingestion policy.
+ */
+public class FeedPolicyAccessor implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ /** failure configuration **/
+ /** continue feed ingestion after a soft (runtime) failure **/
+ public static final String SOFT_FAILURE_CONTINUE = "soft.failure.continue";
+
+ /** log failed tuple to an asterixdb dataset for future reference **/
+ public static final String SOFT_FAILURE_LOG_DATA = "soft.failure.log.data";
+
+ /** continue feed ingestion after loss of one or more machines (hardware failure) **/
+ public static final String HARDWARE_FAILURE_CONTINUE = "hardware.failure.continue";
+
+ /** auto-start a loser feed when the asterixdb instance is restarted **/
+ public static final String CLUSTER_REBOOT_AUTO_RESTART = "cluster.reboot.auto.restart";
+
+ /** framework provides guarantee that each received feed record will be processed through the ingestion pipeline at least once **/
+ public static final String AT_LEAST_ONE_SEMANTICS = "atleast.once.semantics";
+
+ /** flow control configuration **/
+ /** spill excess tuples to disk if an operator cannot process incoming data at its arrival rate **/
+ public static final String SPILL_TO_DISK_ON_CONGESTION = "spill.to.disk.on.congestion";
+
+ /** the maximum size of data (tuples) that can be spilled to disk **/
+ public static final String MAX_SPILL_SIZE_ON_DISK = "max.spill.size.on.disk";
+
+ /** discard tuples altogether if an operator cannot process incoming data at its arrival rate **/
+ public static final String DISCARD_ON_CONGESTION = "discard.on.congestion";
+
+ /** maximum fraction of ingested data that can be discarded **/
+ public static final String MAX_FRACTION_DISCARD = "max.fraction.discard";
+
+ /** maximum end-to-end delay/latency in persisting a tuple through the feed ingestion pipeline **/
+ public static final String MAX_DELAY_RECORD_PERSISTENCE = "max.delay.record.persistence";
+
+ /** rate limit the inflow of tuples in accordance with the maximum capacity of the pipeline **/
+ public static final String THROTTLING_ENABLED = "throttling.enabled";
+
+ /** elasticity **/
+ public static final String ELASTIC = "elastic";
+
+ /** statistics **/
+ public static final String TIME_TRACKING = "time.tracking";
+
+ /** logging of statistics **/
+ public static final String LOGGING_STATISTICS = "logging.statistics";
+
+ public static final long NO_LIMIT = -1;
+
+ private Map<String, String> feedPolicy;
+
+ public Map<String, String> getFeedPolicy() {
+ return feedPolicy;
+ }
+
+ public FeedPolicyAccessor(Map<String, String> feedPolicy) {
+ this.feedPolicy = feedPolicy;
+ }
+
+ public void reset(Map<String, String> feedPolicy) {
+ this.feedPolicy = feedPolicy;
+ }
+
+ /** Failure recover/reporting **/
+
+ public boolean logDataOnSoftFailure() {
+ return getBooleanPropertyValue(SOFT_FAILURE_LOG_DATA, false);
+ }
+
+ public boolean continueOnSoftFailure() {
+ return getBooleanPropertyValue(SOFT_FAILURE_CONTINUE, false);
+ }
+
+ public boolean continueOnHardwareFailure() {
+ return getBooleanPropertyValue(HARDWARE_FAILURE_CONTINUE, false);
+ }
+
+ public boolean autoRestartOnClusterReboot() {
+ return getBooleanPropertyValue(CLUSTER_REBOOT_AUTO_RESTART, false);
+ }
+
+ public boolean atleastOnceSemantics() {
+ return getBooleanPropertyValue(AT_LEAST_ONE_SEMANTICS, false);
+ }
+
+ /** flow control **/
+ public boolean spillToDiskOnCongestion() {
+ return getBooleanPropertyValue(SPILL_TO_DISK_ON_CONGESTION, false);
+ }
+
+ public boolean discardOnCongestion() {
+ return getMaxFractionDiscard() > 0;
+ }
+
+ public boolean throttlingEnabled() {
+ return getBooleanPropertyValue(THROTTLING_ENABLED, false);
+ }
+
+ public long getMaxSpillOnDisk() {
+ return getLongPropertyValue(MAX_SPILL_SIZE_ON_DISK, NO_LIMIT);
+ }
+
+ public float getMaxFractionDiscard() {
+ return getFloatPropertyValue(MAX_FRACTION_DISCARD, 0);
+ }
+
+ public long getMaxDelayRecordPersistence() {
+ return getLongPropertyValue(MAX_DELAY_RECORD_PERSISTENCE, Long.MAX_VALUE);
+ }
+
+ /** Elasticity **/
+ public boolean isElastic() {
+ return getBooleanPropertyValue(ELASTIC, false);
+ }
+
+ /** Statistics **/
+ public boolean isTimeTrackingEnabled() {
+ return getBooleanPropertyValue(TIME_TRACKING, false);
+ }
+
+ /** Logging of statistics **/
+ public boolean isLoggingStatisticsEnabled() {
+ return getBooleanPropertyValue(LOGGING_STATISTICS, false);
+ }
+
+ private boolean getBooleanPropertyValue(String key, boolean defValue) {
+ String v = feedPolicy.get(key);
+ return v == null ? false : Boolean.valueOf(v);
+ }
+
+ private long getLongPropertyValue(String key, long defValue) {
+ String v = feedPolicy.get(key);
+ return v != null ? Long.parseLong(v) : defValue;
+ }
+
+ private float getFloatPropertyValue(String key, float defValue) {
+ String v = feedPolicy.get(key);
+ return v != null ? Float.parseFloat(v) : defValue;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedReport.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedReport.java
deleted file mode 100644
index cda56ae..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedReport.java
+++ /dev/null
@@ -1,116 +0,0 @@
-package edu.uci.ics.asterix.common.feeds;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
-import edu.uci.ics.asterix.common.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-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntime.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntime.java
index 88e1db5..47f373b 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntime.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntime.java
@@ -14,155 +14,57 @@
*/
package edu.uci.ics.asterix.common.feeds;
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.asterix.common.feeds.api.IFeedOperatorOutputSideHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-public class FeedRuntime {
+public class FeedRuntime implements IFeedRuntime {
- public enum FeedRuntimeType {
- INGESTION,
- COMPUTE,
- STORAGE,
- COMMIT
+ /** A unique identifier for the runtime **/
+ protected final FeedRuntimeId runtimeId;
+
+ /** The output frame writer associated with the runtime **/
+ protected IFrameWriter frameWriter;
+
+ /** The pre-processor associated with the runtime **/
+ protected FeedRuntimeInputHandler inputHandler;
+
+ public FeedRuntime(FeedRuntimeId runtimeId, FeedRuntimeInputHandler inputHandler, IFrameWriter frameWriter) {
+ this.runtimeId = runtimeId;
+ this.frameWriter = frameWriter;
+ this.inputHandler = inputHandler;
}
- /** A unique identifier */
- protected final FeedRuntimeId feedRuntimeId;
-
- /** The runtime state: @see FeedRuntimeState */
- protected FeedRuntimeState runtimeState;
-
- public FeedRuntime(FeedConnectionId feedId, int partition, FeedRuntimeType feedRuntimeType) {
- this.feedRuntimeId = new FeedRuntimeId(feedId, feedRuntimeType, partition);
+ public void setFrameWriter(IFeedOperatorOutputSideHandler frameWriter) {
+ this.frameWriter = frameWriter;
}
- public FeedRuntime(FeedConnectionId feedId, int partition, FeedRuntimeType feedRuntimeType, String operandId) {
- this.feedRuntimeId = new FeedRuntimeId(feedId, feedRuntimeType, operandId, partition);
+ @Override
+ public FeedRuntimeId getRuntimeId() {
+ return runtimeId;
}
- public FeedRuntime(FeedConnectionId feedId, int partition, FeedRuntimeType feedRuntimeType, String operandId,
- FeedRuntimeState runtimeState) {
- this.feedRuntimeId = new FeedRuntimeId(feedId, feedRuntimeType, operandId, partition);
- this.runtimeState = runtimeState;
+ @Override
+ public IFrameWriter getFeedFrameWriter() {
+ return frameWriter;
}
@Override
public String toString() {
- return feedRuntimeId + " " + "runtime state present ? " + (runtimeState != null);
+ return runtimeId.toString();
}
- public static class FeedRuntimeState {
-
- private ByteBuffer frame;
- private IFrameWriter frameWriter;
- private Exception exception;
-
- public FeedRuntimeState(ByteBuffer frame, IFrameWriter frameWriter, Exception exception) {
- 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;
- }
-
+ @Override
+ public FeedRuntimeInputHandler getInputHandler() {
+ return inputHandler;
}
- public static class FeedRuntimeId {
-
- public static final String DEFAULT_OPERATION_ID = "N/A";
- private final FeedRuntimeType feedRuntimeType;
- private final String operandId;
- private final FeedConnectionId feedId;
- private final int partition;
- private final int hashCode;
-
- public FeedRuntimeId(FeedConnectionId feedId, FeedRuntimeType runtimeType, String operandId, int partition) {
- this.feedRuntimeType = runtimeType;
- this.operandId = operandId;
- this.feedId = feedId;
- this.partition = partition;
- this.hashCode = (feedId + "[" + partition + "]" + feedRuntimeType).hashCode();
- }
-
- public FeedRuntimeId(FeedConnectionId feedId, FeedRuntimeType runtimeType, int partition) {
- this.feedRuntimeType = runtimeType;
- this.operandId = DEFAULT_OPERATION_ID;
- this.feedId = feedId;
- this.partition = partition;
- this.hashCode = (feedId + "[" + partition + "]" + feedRuntimeType).hashCode();
- }
-
- @Override
- public String toString() {
- return feedId + "[" + partition + "]" + " " + feedRuntimeType + "(" + operandId + ")";
- }
-
- @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.getOperandId().equals(operandId) && oid.getPartition() == partition;
- }
- return false;
- }
-
- public FeedRuntimeType getFeedRuntimeType() {
- return feedRuntimeType;
- }
-
- public FeedConnectionId getFeedId() {
- return feedId;
- }
-
- public String getOperandId() {
- return operandId;
- }
-
- public int getPartition() {
- return partition;
- }
-
+ public Mode getMode() {
+ return inputHandler != null ? inputHandler.getMode() : Mode.PROCESS;
}
- public FeedRuntimeState getRuntimeState() {
- return runtimeState;
- }
-
- public void setRuntimeState(FeedRuntimeState runtimeState) {
- this.runtimeState = runtimeState;
- }
-
- public FeedRuntimeId getFeedRuntimeId() {
- return feedRuntimeId;
+ public void setMode(Mode mode) {
+ this.inputHandler.setMode(mode);
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeId.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeId.java
new file mode 100644
index 0000000..098cc50
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeId.java
@@ -0,0 +1,76 @@
+/*
+ * 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.common.feeds;
+
+import java.io.Serializable;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+
+public class FeedRuntimeId implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final String DEFAULT_OPERAND_ID = "N/A";
+
+ private final FeedRuntimeType runtimeType;
+ private final int partition;
+ private final String operandId;
+
+ public FeedRuntimeId(FeedRuntimeType runtimeType, int partition, String operandId) {
+ this.runtimeType = runtimeType;
+ this.partition = partition;
+ this.operandId = operandId;
+ }
+
+ @Override
+ public String toString() {
+ return runtimeType + "[" + partition + "]" + "{" + operandId + "}";
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof FeedRuntimeId)) {
+ return false;
+ }
+ FeedRuntimeId other = (FeedRuntimeId) o;
+ return (other.getFeedRuntimeType().equals(runtimeType) && other.getOperandId().equals(operandId) && other
+ .getPartition() == partition);
+ }
+
+ @Override
+ public int hashCode() {
+ return toString().hashCode();
+ }
+
+ public FeedRuntimeType getFeedRuntimeType() {
+ return runtimeType;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+
+ public FeedRuntimeType getRuntimeType() {
+ return runtimeType;
+ }
+
+ public String getOperandId() {
+ return operandId;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeInputHandler.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeInputHandler.java
new file mode 100644
index 0000000..8db1c9d
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeInputHandler.java
@@ -0,0 +1,426 @@
+/*
+ * 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.common.feeds;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.feeds.DataBucket.ContentType;
+import edu.uci.ics.asterix.common.feeds.api.IExceptionHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryComponent;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessage;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.Mode;
+import edu.uci.ics.asterix.common.feeds.message.FeedCongestionMessage;
+import edu.uci.ics.asterix.common.feeds.message.ThrottlingEnabledFeedMessage;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+/**
+ * Provides for error-handling and input-side buffering for a feed runtime.
+ */
+public class FeedRuntimeInputHandler implements IFrameWriter {
+
+ private static Logger LOGGER = Logger.getLogger(FeedRuntimeInputHandler.class.getName());
+
+ private final FeedConnectionId connectionId;
+ private final FeedRuntimeId runtimeId;
+ private final FeedPolicyAccessor feedPolicyAccessor;
+ private final boolean bufferingEnabled;
+ private final IExceptionHandler exceptionHandler;
+ private final FeedFrameDiscarder discarder;
+ private final FeedFrameSpiller spiller;
+ private final FeedPolicyAccessor fpa;
+ private final IFeedManager feedManager;
+
+ private IFrameWriter coreOperator;
+ private MonitoredBuffer mBuffer;
+ private DataBucketPool pool;
+ private FrameCollection frameCollection;
+ private Mode mode;
+ private Mode lastMode;
+ private boolean finished;
+ private long nProcessed;
+ private boolean throttlingEnabled;
+
+ private FrameEventCallback frameEventCallback;
+
+ public FeedRuntimeInputHandler(IHyracksTaskContext ctx, FeedConnectionId connectionId, FeedRuntimeId runtimeId, IFrameWriter coreOperator,
+ FeedPolicyAccessor fpa, boolean bufferingEnabled, FrameTupleAccessor fta,
+ RecordDescriptor recordDesc, IFeedManager feedManager, int nPartitions) throws IOException {
+ this.connectionId = connectionId;
+ this.runtimeId = runtimeId;
+ this.coreOperator = coreOperator;
+ this.bufferingEnabled = bufferingEnabled;
+ this.feedPolicyAccessor = fpa;
+ this.spiller = new FeedFrameSpiller(ctx, connectionId, runtimeId, fpa);
+ this.discarder = new FeedFrameDiscarder(ctx, connectionId, runtimeId, fpa, this);
+ this.exceptionHandler = new FeedExceptionHandler(ctx, fta, recordDesc, feedManager, connectionId);
+ this.mode = Mode.PROCESS;
+ this.lastMode = Mode.PROCESS;
+ this.finished = false;
+ this.fpa = fpa;
+ this.feedManager = feedManager;
+ this.pool = (DataBucketPool) feedManager.getFeedMemoryManager().getMemoryComponent(
+ IFeedMemoryComponent.Type.POOL);
+ this.frameCollection = (FrameCollection) feedManager.getFeedMemoryManager().getMemoryComponent(
+ IFeedMemoryComponent.Type.COLLECTION);
+ this.frameEventCallback = new FrameEventCallback(fpa, this, coreOperator);
+ this.mBuffer = MonitoredBuffer.getMonitoredBuffer(ctx, this, coreOperator, fta, recordDesc,
+ feedManager.getFeedMetricCollector(), connectionId, runtimeId, exceptionHandler, frameEventCallback,
+ nPartitions, fpa);
+ this.mBuffer.start();
+ this.throttlingEnabled = false;
+ }
+
+ public synchronized void nextFrame(ByteBuffer frame) throws HyracksDataException {
+ try {
+ switch (mode) {
+ case PROCESS:
+ switch (lastMode) {
+ case SPILL:
+ case POST_SPILL_DISCARD:
+ setMode(Mode.PROCESS_SPILL);
+ processSpilledBacklog();
+ break;
+ case STALL:
+ setMode(Mode.PROCESS_BACKLOG);
+ processBufferredBacklog();
+ break;
+ default:
+ break;
+ }
+ process(frame);
+ break;
+ case PROCESS_BACKLOG:
+ case PROCESS_SPILL:
+ process(frame);
+ break;
+ case SPILL:
+ spill(frame);
+ break;
+ case DISCARD:
+ case POST_SPILL_DISCARD:
+ discard(frame);
+ break;
+ case STALL:
+ switch (runtimeId.getFeedRuntimeType()) {
+ case COLLECT:
+ case COMPUTE_COLLECT:
+ case COMPUTE:
+ case STORE:
+ bufferDataUntilRecovery(frame);
+ break;
+ default:
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Discarding frame during " + mode + " mode " + this.runtimeId);
+ }
+ break;
+ }
+ break;
+ case END:
+ case FAIL:
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Ignoring incoming tuples in " + mode + " mode");
+ }
+ break;
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ private void bufferDataUntilRecovery(ByteBuffer frame) throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Bufferring data until recovery is complete " + this.runtimeId);
+ }
+ if (frameCollection == null) {
+ this.frameCollection = (FrameCollection) feedManager.getFeedMemoryManager().getMemoryComponent(
+ IFeedMemoryComponent.Type.COLLECTION);
+ }
+ if (frameCollection == null) {
+ discarder.processMessage(frame);
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Running low on memory! DISCARDING FRAME ");
+ }
+ } else {
+ boolean success = frameCollection.addFrame(frame);
+ if (!success) {
+ if (fpa.spillToDiskOnCongestion()) {
+ if (frame != null) {
+ spiller.processMessage(frame);
+ } // TODO handle the else case
+ } else {
+ discarder.processMessage(frame);
+ }
+ }
+ }
+ }
+
+ public void reportUnresolvableCongestion() throws HyracksDataException {
+ if (this.runtimeId.getFeedRuntimeType().equals(FeedRuntimeType.COMPUTE)) {
+ FeedCongestionMessage congestionReport = new FeedCongestionMessage(connectionId, runtimeId,
+ mBuffer.getInflowRate(), mBuffer.getOutflowRate(), mode);
+ feedManager.getFeedMessageService().sendMessage(congestionReport);
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Congestion reported " + this.connectionId + " " + this.runtimeId);
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unresolvable congestion at " + this.connectionId + " " + this.runtimeId);
+ }
+ }
+ }
+
+ private void processBufferredBacklog() throws HyracksDataException {
+ try {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Processing backlog " + this.runtimeId);
+ }
+
+ if (frameCollection != null) {
+ Iterator<ByteBuffer> backlog = frameCollection.getFrameCollectionIterator();
+ while (backlog.hasNext()) {
+ process(backlog.next());
+ nProcessed++;
+ }
+ DataBucket bucket = pool.getDataBucket();
+ bucket.setContentType(ContentType.EOSD);
+ bucket.setDesiredReadCount(1);
+ mBuffer.sendMessage(bucket);
+ feedManager.getFeedMemoryManager().releaseMemoryComponent(frameCollection);
+ frameCollection = null;
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ private void processSpilledBacklog() throws HyracksDataException {
+ try {
+ Iterator<ByteBuffer> backlog = spiller.replayData();
+ while (backlog.hasNext()) {
+ process(backlog.next());
+ nProcessed++;
+ }
+ DataBucket bucket = pool.getDataBucket();
+ bucket.setContentType(ContentType.EOSD);
+ bucket.setDesiredReadCount(1);
+ mBuffer.sendMessage(bucket);
+ spiller.reset();
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ protected void process(ByteBuffer frame) throws HyracksDataException {
+ boolean frameProcessed = false;
+ while (!frameProcessed) {
+ try {
+ if (!bufferingEnabled) {
+ coreOperator.nextFrame(frame); // synchronous
+ mBuffer.sendReport(frame);
+ } else {
+ DataBucket bucket = pool.getDataBucket();
+ if (bucket != null) {
+ if (frame != null) {
+ bucket.reset(frame); // created a copy here
+ bucket.setContentType(ContentType.DATA);
+ } else {
+ bucket.setContentType(ContentType.EOD);
+ }
+ bucket.setDesiredReadCount(1);
+ mBuffer.sendMessage(bucket);
+ mBuffer.sendReport(frame);
+ nProcessed++;
+ } else {
+ if (fpa.spillToDiskOnCongestion()) {
+ if (frame != null) {
+ boolean spilled = spiller.processMessage(frame);
+ if (spilled) {
+ setMode(Mode.SPILL);
+ } else {
+ reportUnresolvableCongestion();
+ }
+ }
+ } else if (fpa.discardOnCongestion()) {
+ boolean discarded = discarder.processMessage(frame);
+ if (!discarded) {
+ reportUnresolvableCongestion();
+ }
+ } else if (fpa.throttlingEnabled()) {
+ setThrottlingEnabled(true);
+ } else {
+ reportUnresolvableCongestion();
+ }
+
+ }
+ }
+ frameProcessed = true;
+ } catch (Exception e) {
+ if (feedPolicyAccessor.continueOnSoftFailure()) {
+ frame = exceptionHandler.handleException(e, frame);
+ if (frame == null) {
+ frameProcessed = true;
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Encountered exception! " + e.getMessage()
+ + "Insufficient information, Cannot extract failing tuple");
+ }
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Ingestion policy does not require recovering from tuple. Feed would terminate");
+ }
+ mBuffer.close(false);
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+ }
+
+ private void spill(ByteBuffer frame) throws Exception {
+ boolean success = spiller.processMessage(frame);
+ if (!success) {
+ // limit reached
+ setMode(Mode.POST_SPILL_DISCARD);
+ reportUnresolvableCongestion();
+ }
+ }
+
+ private void discard(ByteBuffer frame) throws Exception {
+ boolean success = discarder.processMessage(frame);
+ if (!success) { // limit reached
+ reportUnresolvableCongestion();
+ }
+ }
+
+ public Mode getMode() {
+ return mode;
+ }
+
+ public synchronized void setMode(Mode mode) {
+ if (mode.equals(this.mode)) {
+ return;
+ }
+ this.lastMode = this.mode;
+ this.mode = mode;
+ if (mode.equals(Mode.END)) {
+ this.close();
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Switched from " + lastMode + " to " + mode + " " + this.runtimeId);
+ }
+ }
+
+ public void close() {
+ if (mBuffer != null) {
+ boolean disableMonitoring = !this.mode.equals(Mode.STALL);
+ if (frameCollection != null) {
+ feedManager.getFeedMemoryManager().releaseMemoryComponent(frameCollection);
+ }
+ if (pool != null) {
+ feedManager.getFeedMemoryManager().releaseMemoryComponent(pool);
+ }
+ mBuffer.close(false, disableMonitoring);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Closed input side handler for " + this.runtimeId + " disabled monitoring "
+ + disableMonitoring + " Mode for runtime " + this.mode);
+ }
+ }
+ }
+
+ public IFrameWriter getCoreOperator() {
+ return coreOperator;
+ }
+
+ public void setCoreOperator(IFrameWriter coreOperator) {
+ this.coreOperator = coreOperator;
+ mBuffer.setFrameWriter(coreOperator);
+ frameEventCallback.setCoreOperator(coreOperator);
+ }
+
+ public boolean isFinished() {
+ return finished;
+ }
+
+ public void setFinished(boolean finished) {
+ this.finished = finished;
+ }
+
+ public long getProcessed() {
+ return nProcessed;
+ }
+
+ public FeedRuntimeId getRuntimeId() {
+ return runtimeId;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ coreOperator.open();
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ coreOperator.fail();
+ }
+
+ public void reset(int nPartitions) {
+ this.mBuffer.setNumberOfPartitions(nPartitions);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Reset number of partitions to " + nPartitions + " for " + this.runtimeId);
+ }
+ if (mBuffer != null) {
+ mBuffer.reset();
+ }
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public IFeedManager getFeedManager() {
+ return feedManager;
+ }
+
+ public MonitoredBuffer getmBuffer() {
+ return mBuffer;
+ }
+
+ public boolean isThrottlingEnabled() {
+ return throttlingEnabled;
+ }
+
+ public void setThrottlingEnabled(boolean throttlingEnabled) {
+ if (this.throttlingEnabled != throttlingEnabled) {
+ this.throttlingEnabled = throttlingEnabled;
+ IFeedMessage throttlingEnabledMesg = new ThrottlingEnabledFeedMessage(connectionId, runtimeId);
+ feedManager.getFeedMessageService().sendMessage(throttlingEnabledMesg);
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Throttling " + throttlingEnabled + " for " + this.connectionId + "[" + runtimeId + "]");
+ }
+ }
+ }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeManager.java
index a68f6b8..d0438f8 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeManager.java
@@ -15,88 +15,40 @@
package edu.uci.ics.asterix.common.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.Set;
import java.util.concurrent.ConcurrentHashMap;
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.common.feeds.FeedRuntime.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedConnectionManager;
public class FeedRuntimeManager {
private static Logger LOGGER = Logger.getLogger(FeedRuntimeManager.class.getName());
- private final FeedConnectionId feedId;
- private final IFeedManager feedManager;
- private SuperFeedManager superFeedManager;
+ private final FeedConnectionId connectionId;
+ private final IFeedConnectionManager connectionManager;
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, IFeedManager feedManager) {
- this.feedId = feedId;
- feedRuntimes = new ConcurrentHashMap<FeedRuntimeId, FeedRuntime>();
- executorService = Executors.newCachedThreadPool();
- feedReportQueue = new LinkedBlockingQueue<String>();
- this.feedManager = feedManager;
+ public FeedRuntimeManager(FeedConnectionId connectionId, IFeedConnectionManager feedConnectionManager) {
+ this.connectionId = connectionId;
+ this.feedRuntimes = new ConcurrentHashMap<FeedRuntimeId, FeedRuntime>();
+ this.executorService = Executors.newCachedThreadPool();
+ this.connectionManager = feedConnectionManager;
}
- public void close(boolean closeAll) throws IOException {
- socketFactory.close();
-
- if (messageService != null) {
- messageService.stop();
+ public void close() throws IOException {
+ if (executorService != null) {
+ executorService.shutdownNow();
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Shut down message services 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);
+ LOGGER.info("Shut down executor service for :" + connectionId);
}
}
-
- 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, feedManager);
- 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) {
@@ -107,17 +59,10 @@
feedRuntimes.put(runtimeId, feedRuntime);
}
- public void deregisterFeedRuntime(FeedRuntimeId runtimeId) {
+ public synchronized 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.deregisterFeed(runtimeId.getFeedId());
- }
- }
+ connectionManager.deregisterFeed(connectionId);
}
}
@@ -125,114 +70,8 @@
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;
- }
-
- 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 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;
+ public Set<FeedRuntimeId> getFeedRuntimes() {
+ return feedRuntimes.keySet();
}
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeReport.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeReport.java
new file mode 100644
index 0000000..868bcc7
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedRuntimeReport.java
@@ -0,0 +1,5 @@
+package edu.uci.ics.asterix.common.feeds;
+
+public class FeedRuntimeReport {
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedStorageStatistics.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedStorageStatistics.java
new file mode 100644
index 0000000..1edf210
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedStorageStatistics.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.asterix.common.feeds;
+
+public class FeedStorageStatistics {
+
+ private long avgPersistenceDelay;
+ private long lastIntakeTimestamp;
+
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedTupleCommitAckMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedTupleCommitAckMessage.java
new file mode 100644
index 0000000..1130905
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedTupleCommitAckMessage.java
@@ -0,0 +1,93 @@
+/*
+ * 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.common.feeds;
+
+import javax.xml.bind.DatatypeConverter;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.message.FeedMessage;
+
+public class FeedTupleCommitAckMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedConnectionId connectionId;
+ private int intakePartition;
+ private int base;
+ private byte[] commitAcks;
+
+ public FeedTupleCommitAckMessage(FeedConnectionId connectionId, int intakePartition, int base, byte[] commitAcks) {
+ super(MessageType.COMMIT_ACK);
+ this.connectionId = connectionId;
+ this.intakePartition = intakePartition;
+ this.base = base;
+ this.commitAcks = commitAcks;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ obj.put(FeedConstants.MessageConstants.INTAKE_PARTITION, intakePartition);
+ obj.put(FeedConstants.MessageConstants.BASE, base);
+ String commitAcksString = DatatypeConverter.printBase64Binary(commitAcks);
+ obj.put(FeedConstants.MessageConstants.COMMIT_ACKS, commitAcksString);
+ return obj;
+ }
+
+ public static FeedTupleCommitAckMessage read(JSONObject obj) throws JSONException {
+ FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
+ obj.getString(FeedConstants.MessageConstants.FEED));
+ FeedConnectionId connectionId = new FeedConnectionId(feedId,
+ obj.getString(FeedConstants.MessageConstants.DATASET));
+ int intakePartition = obj.getInt(FeedConstants.MessageConstants.INTAKE_PARTITION);
+ int base = obj.getInt(FeedConstants.MessageConstants.BASE);
+ String commitAcksString = obj.getString(FeedConstants.MessageConstants.COMMIT_ACKS);
+ byte[] commitAcks = DatatypeConverter.parseBase64Binary(commitAcksString);
+ return new FeedTupleCommitAckMessage(connectionId, intakePartition, base, commitAcks);
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public int getIntakePartition() {
+ return intakePartition;
+ }
+
+ public byte[] getCommitAcks() {
+ return commitAcks;
+ }
+
+ public void reset(int intakePartition, int base, byte[] commitAcks) {
+ this.intakePartition = intakePartition;
+ this.base = base;
+ this.commitAcks = commitAcks;
+ }
+
+ public int getBase() {
+ return base;
+ }
+
+ public void setBase(int base) {
+ this.base = base;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedTupleCommitResponseMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedTupleCommitResponseMessage.java
new file mode 100644
index 0000000..b861e63
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FeedTupleCommitResponseMessage.java
@@ -0,0 +1,76 @@
+/*
+ * 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.common.feeds;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.message.FeedMessage;
+
+public class FeedTupleCommitResponseMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedConnectionId connectionId;
+ private final int intakePartition;
+ private final int maxWindowAcked;
+
+ public FeedTupleCommitResponseMessage(FeedConnectionId connectionId, int intakePartition, int maxWindowAcked) {
+ super(MessageType.COMMIT_ACK_RESPONSE);
+ this.connectionId = connectionId;
+ this.intakePartition = intakePartition;
+ this.maxWindowAcked = maxWindowAcked;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ obj.put(FeedConstants.MessageConstants.INTAKE_PARTITION, intakePartition);
+ obj.put(FeedConstants.MessageConstants.MAX_WINDOW_ACKED, maxWindowAcked);
+ return obj;
+ }
+
+ @Override
+ public String toString() {
+ return connectionId + "[" + intakePartition + "]" + "(" + maxWindowAcked + ")";
+ }
+
+ public static FeedTupleCommitResponseMessage read(JSONObject obj) throws JSONException {
+ FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
+ obj.getString(FeedConstants.MessageConstants.FEED));
+ FeedConnectionId connectionId = new FeedConnectionId(feedId,
+ obj.getString(FeedConstants.MessageConstants.DATASET));
+ int intakePartition = obj.getInt(FeedConstants.MessageConstants.INTAKE_PARTITION);
+ int maxWindowAcked = obj.getInt(FeedConstants.MessageConstants.MAX_WINDOW_ACKED);
+ return new FeedTupleCommitResponseMessage(connectionId, intakePartition, maxWindowAcked);
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public int getMaxWindowAcked() {
+ return maxWindowAcked;
+ }
+
+ public int getIntakePartition() {
+ return intakePartition;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FrameCollection.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FrameCollection.java
new file mode 100644
index 0000000..611f613
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FrameCollection.java
@@ -0,0 +1,97 @@
+/*
+ * 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.common.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryComponent;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryManager;
+
+/**
+ * Represents an expandable collection of frames.
+ */
+public class FrameCollection implements IFeedMemoryComponent {
+
+ /** A unique identifier for the feed memory component **/
+ private final int componentId;
+
+ /** A collection of frames (each being a ByteBuffer) **/
+ private final List<ByteBuffer> frames = new LinkedList<ByteBuffer>();
+
+ /** The permitted maximum size, the collection may grow to **/
+ private int maxSize;
+
+ /** The {@link IFeedMemoryManager} for the NodeController **/
+ private final IFeedMemoryManager memoryManager;
+
+ public FrameCollection(int componentId, IFeedMemoryManager memoryManager, int maxSize) {
+ this.componentId = componentId;
+ this.maxSize = maxSize;
+ this.memoryManager = memoryManager;
+ }
+
+ public boolean addFrame(ByteBuffer frame) {
+ if (frames.size() == maxSize) {
+ boolean expansionGranted = memoryManager.expandMemoryComponent(this);
+ if (!expansionGranted) {
+ return false;
+ }
+ }
+ ByteBuffer storageBuffer = ByteBuffer.allocate(frame.capacity());
+ storageBuffer.put(frame);
+ frames.add(storageBuffer);
+ storageBuffer.flip();
+ return true;
+ }
+
+ public Iterator<ByteBuffer> getFrameCollectionIterator() {
+ return frames.iterator();
+ }
+
+ @Override
+ public int getTotalAllocation() {
+ return frames.size();
+ }
+
+ @Override
+ public Type getType() {
+ return Type.COLLECTION;
+ }
+
+ @Override
+ public int getComponentId() {
+ return componentId;
+ }
+
+ @Override
+ public void expand(int delta) {
+ maxSize = maxSize + delta;
+ }
+
+ @Override
+ public void reset() {
+ frames.clear();
+ maxSize = IFeedMemoryManager.START_COLLECTION_SIZE;
+ }
+
+ @Override
+ public String toString() {
+ return "FrameCollection" + "[" + componentId + "]" + "(" + frames.size() + "/" + maxSize + ")";
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FrameDistributor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FrameDistributor.java
new file mode 100644
index 0000000..937e8f8
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FrameDistributor.java
@@ -0,0 +1,359 @@
+/*
+ * 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.common.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryComponent.Type;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+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.FrameTupleAccessor;
+
+public class FrameDistributor {
+
+ private static final Logger LOGGER = Logger.getLogger(FrameDistributor.class.getName());
+
+ private static final long MEMORY_AVAILABLE_POLL_PERIOD = 1000; // 1 second
+
+ private final IHyracksTaskContext ctx;
+ private final FeedId feedId;
+ private final FeedRuntimeType feedRuntimeType;
+ private final int partition;
+ private final IFeedMemoryManager memoryManager;
+ private final boolean enableSynchronousTransfer;
+ /** A map storing the registered frame readers ({@code FeedFrameCollector}. **/
+ private final Map<IFrameWriter, FeedFrameCollector> registeredCollectors;
+ private final FrameTupleAccessor fta;
+
+ private DataBucketPool pool;
+ private DistributionMode distributionMode;
+ private boolean spillToDiskRequired = false;
+
+ public enum DistributionMode {
+ /**
+ * A single feed frame collector is registered for receiving tuples.
+ * Tuple is sent via synchronous call, that is no buffering is involved
+ **/
+ SINGLE,
+
+ /**
+ * Multiple feed frame collectors are concurrently registered for
+ * receiving tuples.
+ **/
+ SHARED,
+
+ /**
+ * Feed tuples are not being processed, irrespective of # of registered
+ * feed frame collectors.
+ **/
+ INACTIVE
+ }
+
+ public FrameDistributor(IHyracksTaskContext ctx, FeedId feedId, FeedRuntimeType feedRuntimeType, int partition,
+ boolean enableSynchronousTransfer, IFeedMemoryManager memoryManager, FrameTupleAccessor fta)
+ throws HyracksDataException {
+ this.ctx = ctx;
+ this.feedId = feedId;
+ this.feedRuntimeType = feedRuntimeType;
+ this.partition = partition;
+ this.memoryManager = memoryManager;
+ this.enableSynchronousTransfer = enableSynchronousTransfer;
+ this.registeredCollectors = new HashMap<IFrameWriter, FeedFrameCollector>();
+ this.distributionMode = DistributionMode.INACTIVE;
+ this.fta = fta;
+ }
+
+ public void notifyEndOfFeed() {
+ DataBucket bucket = getDataBucket();
+ if (bucket != null) {
+ sendEndOfFeedDataBucket(bucket);
+ } else {
+ while (bucket == null) {
+ try {
+ Thread.sleep(MEMORY_AVAILABLE_POLL_PERIOD);
+ bucket = getDataBucket();
+ } catch (InterruptedException e) {
+ break;
+ }
+ }
+ if (bucket != null) {
+ sendEndOfFeedDataBucket(bucket);
+ }
+ }
+ }
+
+ private void sendEndOfFeedDataBucket(DataBucket bucket) {
+ bucket.setContentType(DataBucket.ContentType.EOD);
+ nextBucket(bucket);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("End of feed data packet sent " + this.feedId);
+ }
+ }
+
+ public synchronized void registerFrameCollector(FeedFrameCollector frameCollector) {
+ DistributionMode currentMode = distributionMode;
+ switch (distributionMode) {
+ case INACTIVE:
+ if (!enableSynchronousTransfer) {
+ pool = (DataBucketPool) memoryManager.getMemoryComponent(Type.POOL);
+ frameCollector.start();
+ }
+ registeredCollectors.put(frameCollector.getFrameWriter(), frameCollector);
+ setMode(DistributionMode.SINGLE);
+ break;
+ case SINGLE:
+ pool = (DataBucketPool) memoryManager.getMemoryComponent(Type.POOL);
+ registeredCollectors.put(frameCollector.getFrameWriter(), frameCollector);
+ for (FeedFrameCollector reader : registeredCollectors.values()) {
+ reader.start();
+ }
+ setMode(DistributionMode.SHARED);
+ break;
+ case SHARED:
+ frameCollector.start();
+ registeredCollectors.put(frameCollector.getFrameWriter(), frameCollector);
+ break;
+ }
+ evaluateIfSpillIsEnabled();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Switching to " + distributionMode + " mode from " + currentMode + " mode " + " Feed id "
+ + feedId);
+ }
+ }
+
+ public synchronized void deregisterFrameCollector(FeedFrameCollector frameCollector) {
+ switch (distributionMode) {
+ case INACTIVE:
+ throw new IllegalStateException("Invalid attempt to deregister frame collector in " + distributionMode
+ + " mode.");
+ case SHARED:
+ frameCollector.closeCollector();
+ registeredCollectors.remove(frameCollector.getFrameWriter());
+ int nCollectors = registeredCollectors.size();
+ if (nCollectors == 1) {
+ FeedFrameCollector loneCollector = registeredCollectors.values().iterator().next();
+ setMode(DistributionMode.SINGLE);
+ loneCollector.setState(FeedFrameCollector.State.TRANSITION);
+ loneCollector.closeCollector();
+ memoryManager.releaseMemoryComponent(pool);
+ evaluateIfSpillIsEnabled();
+ } else {
+ if (!spillToDiskRequired) {
+ evaluateIfSpillIsEnabled();
+ }
+ }
+ break;
+ case SINGLE:
+ frameCollector.closeCollector();
+ setMode(DistributionMode.INACTIVE);
+ spillToDiskRequired = false;
+ break;
+
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Deregistered frame reader" + frameCollector + " from feed distributor for " + feedId);
+ }
+ }
+
+ public void evaluateIfSpillIsEnabled() {
+ if (!spillToDiskRequired) {
+ for (FeedFrameCollector collector : registeredCollectors.values()) {
+ spillToDiskRequired = spillToDiskRequired
+ || collector.getFeedPolicyAccessor().spillToDiskOnCongestion();
+ if (spillToDiskRequired) {
+ break;
+ }
+ }
+ }
+ }
+
+ public boolean deregisterFrameCollector(IFrameWriter frameWriter) {
+ FeedFrameCollector collector = registeredCollectors.get(frameWriter);
+ if (collector != null) {
+ deregisterFrameCollector(collector);
+ return true;
+ }
+ return false;
+ }
+
+ public synchronized void setMode(DistributionMode mode) {
+ this.distributionMode = mode;
+ }
+
+ public boolean isRegistered(IFrameWriter writer) {
+ return registeredCollectors.get(writer) != null;
+ }
+
+ public synchronized void nextFrame(ByteBuffer frame) throws HyracksDataException {
+ switch (distributionMode) {
+ case INACTIVE:
+ break;
+ case SINGLE:
+ FeedFrameCollector collector = registeredCollectors.values().iterator().next();
+ switch (collector.getState()) {
+ case HANDOVER:
+ case ACTIVE:
+ if (enableSynchronousTransfer) {
+ collector.nextFrame(frame); // processing is synchronous
+ } else {
+ handleDataBucket(frame);
+ }
+ break;
+ case TRANSITION:
+ handleDataBucket(frame);
+ break;
+ case FINISHED:
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Discarding fetched tuples, feed has ended [" + registeredCollectors.get(0)
+ + "]" + " Feed Id " + feedId + " frame distributor " + this.getFeedRuntimeType());
+ }
+ registeredCollectors.remove(0);
+ break;
+ }
+ break;
+ case SHARED:
+ handleDataBucket(frame);
+ break;
+ }
+ }
+
+ private void nextBucket(DataBucket bucket) {
+ for (FeedFrameCollector collector : registeredCollectors.values()) {
+ collector.sendMessage(bucket); // asynchronous call
+ }
+ }
+
+ private void handleDataBucket(ByteBuffer frame) throws HyracksDataException {
+ DataBucket bucket = getDataBucket();
+ if (bucket == null) {
+ handleFrameDuringMemoryCongestion(frame);
+ } else {
+ bucket.reset(frame);
+ bucket.setDesiredReadCount(registeredCollectors.size());
+ nextBucket(bucket);
+ }
+ }
+
+ private void handleFrameDuringMemoryCongestion(ByteBuffer frame) throws HyracksDataException {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to allocate memory, will evaluate the need to spill");
+ }
+ // wait till memory is available
+ }
+
+ private DataBucket getDataBucket() {
+ DataBucket bucket = null;
+ if (pool != null) {
+ bucket = pool.getDataBucket();
+ if (bucket != null) {
+ bucket.setDesiredReadCount(registeredCollectors.size());
+ return bucket;
+ } else {
+ return null;
+ }
+ }
+ return null;
+ }
+
+ public DistributionMode getMode() {
+ return distributionMode;
+ }
+
+ public void close() {
+ switch (distributionMode) {
+ case INACTIVE:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("FrameDistributor is " + distributionMode);
+ }
+ break;
+ case SINGLE:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Disconnecting single frame reader in " + distributionMode + " mode " + " for feedId "
+ + feedId + " " + this.feedRuntimeType);
+ }
+ setMode(DistributionMode.INACTIVE);
+ if (!enableSynchronousTransfer) {
+ notifyEndOfFeed(); // send EOD Data Bucket
+ waitForCollectorsToFinish();
+ }
+ registeredCollectors.values().iterator().next().disconnect();
+ break;
+ case SHARED:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Signalling End Of Feed; currently operating in " + distributionMode + " mode");
+ }
+ notifyEndOfFeed(); // send EOD Data Bucket
+ waitForCollectorsToFinish();
+ break;
+ }
+ }
+
+ private void waitForCollectorsToFinish() {
+ synchronized (registeredCollectors.values()) {
+ while (!allCollectorsFinished()) {
+ try {
+ registeredCollectors.values().wait();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+
+ private boolean allCollectorsFinished() {
+ boolean allFinished = true;
+ for (FeedFrameCollector collector : registeredCollectors.values()) {
+ allFinished = allFinished && collector.getState().equals(FeedFrameCollector.State.FINISHED);
+ }
+ return allFinished;
+ }
+
+ public Collection<FeedFrameCollector> getRegisteredCollectors() {
+ return registeredCollectors.values();
+ }
+
+ public Map<IFrameWriter, FeedFrameCollector> getRegisteredReaders() {
+ return registeredCollectors;
+ }
+
+ public FeedId getFeedId() {
+ return feedId;
+ }
+
+ public DistributionMode getDistributionMode() {
+ return distributionMode;
+ }
+
+ public FeedRuntimeType getFeedRuntimeType() {
+ return feedRuntimeType;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+
+ public FrameTupleAccessor getFta() {
+ return fta;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FrameEventCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FrameEventCallback.java
new file mode 100644
index 0000000..780a332
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/FrameEventCallback.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2009-2014 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.common.feeds;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.Mode;
+import edu.uci.ics.asterix.common.feeds.api.IFrameEventCallback;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class FrameEventCallback implements IFrameEventCallback {
+
+ private static final Logger LOGGER = Logger.getLogger(FrameEventCallback.class.getName());
+
+ private final FeedPolicyAccessor fpa;
+ private final FeedRuntimeInputHandler inputSideHandler;
+ private IFrameWriter coreOperator;
+
+ public FrameEventCallback(FeedPolicyAccessor fpa, FeedRuntimeInputHandler inputSideHandler,
+ IFrameWriter coreOperator) {
+ this.fpa = fpa;
+ this.inputSideHandler = inputSideHandler;
+ this.coreOperator = coreOperator;
+ }
+
+ @Override
+ public void frameEvent(FrameEvent event) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Frame Event for " + inputSideHandler.getRuntimeId() + " " + event);
+ }
+ if (!event.equals(FrameEvent.FINISHED_PROCESSING_SPILLAGE)
+ && inputSideHandler.getMode().equals(Mode.PROCESS_SPILL)) {
+ return;
+ }
+ switch (event) {
+ case PENDING_WORK_THRESHOLD_REACHED:
+ if (fpa.spillToDiskOnCongestion()) {
+ inputSideHandler.setMode(Mode.SPILL);
+ } else if (fpa.discardOnCongestion()) {
+ inputSideHandler.setMode(Mode.DISCARD);
+ } else if (fpa.throttlingEnabled()) {
+ inputSideHandler.setThrottlingEnabled(true);
+ } else {
+ try {
+ inputSideHandler.reportUnresolvableCongestion();
+ } catch (HyracksDataException e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to report congestion!!!");
+ }
+ }
+ }
+ break;
+ case FINISHED_PROCESSING:
+ inputSideHandler.setFinished(true);
+ synchronized (coreOperator) {
+ coreOperator.notifyAll();
+ }
+ break;
+ case PENDING_WORK_DONE:
+ switch (inputSideHandler.getMode()) {
+ case SPILL:
+ case DISCARD:
+ case POST_SPILL_DISCARD:
+ inputSideHandler.setMode(Mode.PROCESS);
+ break;
+ default:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Received " + event + " ignoring as operating in " + inputSideHandler.getMode());
+ }
+ }
+ break;
+ case FINISHED_PROCESSING_SPILLAGE:
+ inputSideHandler.setMode(Mode.PROCESS);
+ break;
+ default:
+ break;
+ }
+ }
+
+ public void setCoreOperator(IFrameWriter coreOperator) {
+ this.coreOperator = coreOperator;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IFeedManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IFeedManager.java
deleted file mode 100644
index 6cdc45c..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IFeedManager.java
+++ /dev/null
@@ -1,101 +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.common.feeds;
-
-import java.io.IOException;
-import java.util.concurrent.ExecutorService;
-
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeId;
-
-/**
- * Handle (de)registration of feeds for delivery of control messages.
- */
-public interface IFeedManager {
-
- public static final long SOCKET_CONNECT_TIMEOUT = 5000;
-
- /**
- * Returns the executor service associated with the feed.
- *
- * @param feedId
- * @return
- */
- public ExecutorService getFeedExecutorService(FeedConnectionId feedId);
-
- /**
- * Allows registration of a feedRuntime.
- *
- * @param feedRuntime
- * @throws Exception
- */
- public void registerFeedRuntime(FeedRuntime feedRuntime) throws Exception;
-
- /**
- * Allows de-registration of a feed runtime.
- *
- * @param feedRuntimeId
- */
- public void deRegisterFeedRuntime(FeedRuntimeId feedRuntimeId);
-
- /**
- * Obtain feed runtime corresponding to a feedRuntimeId
- *
- * @param feedRuntimeId
- * @return
- */
- public FeedRuntime getFeedRuntime(FeedRuntimeId feedRuntimeId);
-
- /**
- * Register the Super Feed Manager associated witht a feed.
- *
- * @param feedId
- * @param sfm
- * @throws Exception
- */
- public void registerSuperFeedManager(FeedConnectionId feedId, SuperFeedManager sfm) throws Exception;
-
- /**
- * Obtain a handle to the Super Feed Manager associated with the feed.
- *
- * @param feedId
- * @return
- */
- public SuperFeedManager getSuperFeedManager(FeedConnectionId feedId);
-
- /**
- * De-register a feed
- *
- * @param feedId
- * @throws IOException
- */
- void deregisterFeed(FeedConnectionId feedId);
-
- /**
- * Obtain the feed runtime manager associated with a feed.
- *
- * @param feedId
- * @return
- */
- public FeedRuntimeManager getFeedRuntimeManager(FeedConnectionId feedId);
-
- /**
- * Obtain a handle to the feed Message service associated with a feed.
- *
- * @param feedId
- * @return
- */
- public FeedMessageService getFeedMessageService(FeedConnectionId feedId);
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IFramePostProcessor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IFramePostProcessor.java
new file mode 100644
index 0000000..3bda2db
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IFramePostProcessor.java
@@ -0,0 +1,10 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public interface IFramePostProcessor {
+
+ public void postProcessFrame(ByteBuffer frame, FrameTupleAccessor frameAccessor);
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IFramePreprocessor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IFramePreprocessor.java
new file mode 100644
index 0000000..468c13c
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IFramePreprocessor.java
@@ -0,0 +1,8 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.nio.ByteBuffer;
+
+public interface IFramePreprocessor {
+
+ public void preProcess(ByteBuffer frame) throws Exception;
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IngestionRuntime.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IngestionRuntime.java
new file mode 100644
index 0000000..b7eb3b8
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IngestionRuntime.java
@@ -0,0 +1,71 @@
+/*
+ * 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.common.feeds;
+
+import java.util.logging.Level;
+
+import edu.uci.ics.asterix.common.feeds.api.IAdapterRuntimeManager;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class IngestionRuntime extends SubscribableRuntime {
+
+ private final IAdapterRuntimeManager adapterRuntimeManager;
+
+ public IngestionRuntime(FeedId feedId, FeedRuntimeId runtimeId, DistributeFeedFrameWriter feedWriter,
+ RecordDescriptor recordDesc, IAdapterRuntimeManager adaptorRuntimeManager) {
+ super(feedId, runtimeId, null, feedWriter, recordDesc);
+ this.adapterRuntimeManager = adaptorRuntimeManager;
+ }
+
+ public void subscribeFeed(FeedPolicyAccessor fpa, CollectionRuntime collectionRuntime) throws Exception {
+ FeedFrameCollector reader = dWriter.subscribeFeed(fpa, collectionRuntime.getInputHandler(),
+ collectionRuntime.getConnectionId());
+ collectionRuntime.setFrameCollector(reader);
+
+ if (dWriter.getDistributionMode().equals(FrameDistributor.DistributionMode.SINGLE)) {
+ adapterRuntimeManager.start();
+ }
+ subscribers.add(collectionRuntime);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Subscribed feed collection [" + collectionRuntime + "] to " + this);
+ }
+ }
+
+ public void unsubscribeFeed(CollectionRuntime collectionRuntime) throws Exception {
+ dWriter.unsubscribeFeed(collectionRuntime.getInputHandler());
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Unsubscribed feed collection [" + collectionRuntime + "] from " + this);
+ }
+ if (dWriter.getDistributionMode().equals(FrameDistributor.DistributionMode.INACTIVE)) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Stopping adapter for " + this + " as no more registered collectors");
+ }
+ adapterRuntimeManager.stop();
+ }
+ subscribers.remove(collectionRuntime);
+ }
+
+ public void endOfFeed() {
+ dWriter.notifyEndOfFeed();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Notified End Of Feed [" + this + "]");
+ }
+ }
+
+ public IAdapterRuntimeManager getAdapterRuntimeManager() {
+ return adapterRuntimeManager;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IntakePartitionStatistics.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IntakePartitionStatistics.java
new file mode 100644
index 0000000..656797e
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IntakePartitionStatistics.java
@@ -0,0 +1,27 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.util.BitSet;
+
+public class IntakePartitionStatistics {
+
+ public static int ACK_WINDOW_SIZE = 1024;
+ private int partition;
+ private int base;
+ private BitSet bitSet;
+
+ public IntakePartitionStatistics(int partition, int base) {
+ this.partition = partition;
+ this.base = base;
+ this.bitSet = new BitSet(ACK_WINDOW_SIZE);
+ }
+
+ public void ackRecordId(int recordId) {
+ int posIndexWithinBase = recordId % ACK_WINDOW_SIZE;
+ this.bitSet.set(posIndexWithinBase);
+ }
+
+ public byte[] getAckInfo() {
+ return bitSet.toByteArray();
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IntakeSideMonitoredBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IntakeSideMonitoredBuffer.java
new file mode 100644
index 0000000..fdd6ec4
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/IntakeSideMonitoredBuffer.java
@@ -0,0 +1,56 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import edu.uci.ics.asterix.common.feeds.api.IExceptionHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector;
+import edu.uci.ics.asterix.common.feeds.api.IFrameEventCallback;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class IntakeSideMonitoredBuffer extends MonitoredBuffer {
+
+ public IntakeSideMonitoredBuffer(IHyracksTaskContext ctx, FeedRuntimeInputHandler inputHandler, IFrameWriter frameWriter,
+ FrameTupleAccessor fta, RecordDescriptor recordDesc, IFeedMetricCollector metricCollector,
+ FeedConnectionId connectionId, FeedRuntimeId runtimeId, IExceptionHandler exceptionHandler,
+ IFrameEventCallback callback, int nPartitions, FeedPolicyAccessor policyAccessor) {
+ super(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector, connectionId, runtimeId,
+ exceptionHandler, callback, nPartitions, policyAccessor);
+ }
+
+ @Override
+ protected boolean monitorProcessingRate() {
+ return false;
+ }
+
+ @Override
+ protected boolean logInflowOutflowRate() {
+ return false;
+ }
+
+ @Override
+ protected IFramePreprocessor getFramePreProcessor() {
+ return null;
+ }
+
+ @Override
+ protected IFramePostProcessor getFramePostProcessor() {
+ return null;
+ }
+
+ @Override
+ protected boolean monitorInputQueueLength() {
+ return false;
+ }
+
+ @Override
+ protected boolean reportOutflowRate() {
+ return false;
+ }
+
+ @Override
+ protected boolean reportInflowRate() {
+ return true;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MessageListener.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MessageListener.java
index 7beb212..2e21ea7 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MessageListener.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MessageListener.java
@@ -43,15 +43,12 @@
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);
}
@@ -62,6 +59,8 @@
private final LinkedBlockingQueue<String> outbox;
private ServerSocket server;
+ private static final char EOL = (char) "\n".getBytes()[0];
+
public MessageListenerServer(int port, LinkedBlockingQueue<String> outbox) {
this.port = port;
this.outbox = outbox;
@@ -77,7 +76,6 @@
@Override
public void run() {
- char EOL = (char) "\n".getBytes()[0];
Socket client = null;
try {
server = new ServerSocket(port);
@@ -121,59 +119,4 @@
}
- 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-common/src/main/java/edu/uci/ics/asterix/common/feeds/MessageReceiver.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MessageReceiver.java
new file mode 100644
index 0000000..2cb1066
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MessageReceiver.java
@@ -0,0 +1,107 @@
+/*
+ * 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.common.feeds;
+
+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.common.feeds.api.IMessageReceiver;
+
+public abstract class MessageReceiver<T> implements IMessageReceiver<T> {
+
+ protected static final Logger LOGGER = Logger.getLogger(MessageReceiver.class.getName());
+
+ protected final LinkedBlockingQueue<T> inbox;
+ protected ExecutorService executor;
+
+ public MessageReceiver() {
+ inbox = new LinkedBlockingQueue<T>();
+ }
+
+ public abstract void processMessage(T message) throws Exception;
+
+ @Override
+ public void start() {
+ executor = Executors.newSingleThreadExecutor();
+ executor.execute(new MessageReceiverRunnable<T>(this));
+ }
+
+ @Override
+ public synchronized void sendMessage(T message) {
+ inbox.add(message);
+ }
+
+ @Override
+ public void close(boolean processPending) {
+ if (executor != null) {
+ executor.shutdown();
+ executor = null;
+ if (processPending) {
+ flushPendingMessages();
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Will discard the pending frames " + inbox.size());
+ }
+ }
+ }
+ }
+
+ private static class MessageReceiverRunnable<T> implements Runnable {
+
+ private final LinkedBlockingQueue<T> inbox;
+ private final MessageReceiver<T> messageReceiver;
+
+ public MessageReceiverRunnable(MessageReceiver<T> messageReceiver) {
+ this.inbox = messageReceiver.inbox;
+ this.messageReceiver = messageReceiver;
+ }
+
+ @Override
+ public void run() {
+ while (true) {
+ try {
+ T message = inbox.take();
+ messageReceiver.processMessage(message);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+
+ protected void flushPendingMessages() {
+ while (!inbox.isEmpty()) {
+ T message = null;
+ try {
+ message = inbox.take();
+ processMessage(message);
+ } catch (InterruptedException ie) {
+ // ignore exception but break from the loop
+ break;
+ } catch (Exception e) {
+ e.printStackTrace();
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Exception " + e + " in processing message " + message);
+ }
+ }
+ }
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MonitoredBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MonitoredBuffer.java
new file mode 100644
index 0000000..90e340f
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MonitoredBuffer.java
@@ -0,0 +1,386 @@
+/*
+ * Copyright 2009-2014 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.common.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.logging.Level;
+
+import edu.uci.ics.asterix.common.feeds.MonitoredBufferTimerTasks.LogInputOutputRateTask;
+import edu.uci.ics.asterix.common.feeds.MonitoredBufferTimerTasks.MonitorInputQueueLengthTimerTask;
+import edu.uci.ics.asterix.common.feeds.MonitoredBufferTimerTasks.MonitoreProcessRateTimerTask;
+import edu.uci.ics.asterix.common.feeds.MonitoredBufferTimerTasks.MonitoredBufferStorageTimerTask;
+import edu.uci.ics.asterix.common.feeds.api.IExceptionHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector.MetricType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector.ValueType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.Mode;
+import edu.uci.ics.asterix.common.feeds.api.IFrameEventCallback;
+import edu.uci.ics.asterix.common.feeds.api.IFrameEventCallback.FrameEvent;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public abstract class MonitoredBuffer extends MessageReceiver<DataBucket> {
+
+ protected static final long LOG_INPUT_OUTPUT_RATE_FREQUENCY = 5000; // 5 seconds
+ protected static final long INPUT_QUEUE_MEASURE_FREQUENCY = 1000; // 1 second
+ protected static final long PROCESSING_RATE_MEASURE_FREQUENCY = 10000; // 10 seconds
+
+ protected static final int PROCESS_RATE_REFRESH = 2; // refresh processing rate every 10th frame
+
+ protected final IHyracksTaskContext ctx;
+ protected final FeedConnectionId connectionId;
+ protected final FeedRuntimeId runtimeId;
+ protected final FrameTupleAccessor inflowFta;
+ protected final FrameTupleAccessor outflowFta;
+ protected final FeedRuntimeInputHandler inputHandler;
+ protected final IFrameEventCallback callback;
+ protected final Timer timer;
+ private final RecordDescriptor recordDesc;
+ private final IExceptionHandler exceptionHandler;
+ protected final FeedPolicyAccessor policyAccessor;
+ protected int nPartitions;
+
+ private IFrameWriter frameWriter;
+ protected IFeedMetricCollector metricCollector;
+ protected boolean monitorProcessingRate = false;
+ protected boolean monitorInputQueueLength = false;
+ protected boolean logInflowOutflowRate = false;
+ protected boolean reportOutflowRate = false;
+ protected boolean reportInflowRate = false;
+
+ protected int inflowReportSenderId = -1;
+ protected int outflowReportSenderId = -1;
+ protected TimerTask monitorInputQueueLengthTask;
+ protected TimerTask processingRateTask;
+ protected TimerTask logInflowOutflowRateTask;
+ protected MonitoredBufferStorageTimerTask storageTimeTrackingRateTask;
+ protected StorageFrameHandler storageFromeHandler;
+
+ protected int processingRate = -1;
+ protected int frameCount = 0;
+ private long avgDelayPersistence = 0;
+ private boolean active;
+ private Map<Integer, Long> tupleTimeStats;
+ IFramePostProcessor postProcessor = null;
+ IFramePreprocessor preProcessor = null;
+
+ public static MonitoredBuffer getMonitoredBuffer(IHyracksTaskContext ctx, FeedRuntimeInputHandler inputHandler,
+ IFrameWriter frameWriter, FrameTupleAccessor fta, RecordDescriptor recordDesc,
+ IFeedMetricCollector metricCollector, FeedConnectionId connectionId, FeedRuntimeId runtimeId,
+ IExceptionHandler exceptionHandler, IFrameEventCallback callback, int nPartitions,
+ FeedPolicyAccessor policyAccessor) {
+ switch (runtimeId.getFeedRuntimeType()) {
+ case COMPUTE:
+ return new ComputeSideMonitoredBuffer(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector,
+ connectionId, runtimeId, exceptionHandler, callback, nPartitions, policyAccessor);
+ case STORE:
+ return new StorageSideMonitoredBuffer(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector,
+ connectionId, runtimeId, exceptionHandler, callback, nPartitions, policyAccessor);
+ case COLLECT:
+ return new IntakeSideMonitoredBuffer(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector,
+ connectionId, runtimeId, exceptionHandler, callback, nPartitions, policyAccessor);
+ default:
+ return new BasicMonitoredBuffer(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector,
+ connectionId, runtimeId, exceptionHandler, callback, nPartitions, policyAccessor);
+ }
+ }
+
+ protected MonitoredBuffer(IHyracksTaskContext ctx, FeedRuntimeInputHandler inputHandler, IFrameWriter frameWriter,
+ FrameTupleAccessor fta, RecordDescriptor recordDesc, IFeedMetricCollector metricCollector,
+ FeedConnectionId connectionId, FeedRuntimeId runtimeId, IExceptionHandler exceptionHandler,
+ IFrameEventCallback callback, int nPartitions, FeedPolicyAccessor policyAccessor) {
+ this.ctx = ctx;
+ this.connectionId = connectionId;
+ this.frameWriter = frameWriter;
+ this.inflowFta = new FrameTupleAccessor(recordDesc);
+ this.outflowFta = new FrameTupleAccessor(recordDesc);
+ this.runtimeId = runtimeId;
+ this.metricCollector = metricCollector;
+ this.exceptionHandler = exceptionHandler;
+ this.callback = callback;
+ this.inputHandler = inputHandler;
+ this.timer = new Timer();
+ this.recordDesc = recordDesc;
+ this.policyAccessor = policyAccessor;
+ this.nPartitions = nPartitions;
+ this.active = true;
+ initializeMonitoring();
+ }
+
+ protected abstract boolean monitorProcessingRate();
+
+ protected abstract boolean logInflowOutflowRate();
+
+ protected abstract boolean reportOutflowRate();
+
+ protected abstract boolean reportInflowRate();
+
+ protected abstract boolean monitorInputQueueLength();
+
+ protected abstract IFramePreprocessor getFramePreProcessor();
+
+ protected abstract IFramePostProcessor getFramePostProcessor();
+
+ protected void initializeMonitoring() {
+ monitorProcessingRate = monitorProcessingRate();
+ monitorInputQueueLength = monitorInputQueueLength();
+ reportInflowRate = reportInflowRate();
+ reportOutflowRate = reportOutflowRate();
+ logInflowOutflowRate = policyAccessor.isLoggingStatisticsEnabled() || logInflowOutflowRate();
+
+ if (monitorProcessingRate && policyAccessor.isElastic()) { // check possibility to scale in
+ this.processingRateTask = new MonitoreProcessRateTimerTask(this, inputHandler.getFeedManager(),
+ connectionId, nPartitions);
+ this.timer.scheduleAtFixedRate(processingRateTask, 0, PROCESSING_RATE_MEASURE_FREQUENCY);
+ }
+
+ if (monitorInputQueueLength
+ && (policyAccessor.isElastic() || policyAccessor.throttlingEnabled()
+ || policyAccessor.spillToDiskOnCongestion() || policyAccessor.discardOnCongestion())) {
+ this.monitorInputQueueLengthTask = new MonitorInputQueueLengthTimerTask(this, callback);
+ this.timer.scheduleAtFixedRate(monitorInputQueueLengthTask, 0, INPUT_QUEUE_MEASURE_FREQUENCY);
+ }
+
+ if (logInflowOutflowRate || reportInflowRate || reportOutflowRate) {
+ this.logInflowOutflowRateTask = new LogInputOutputRateTask(this, logInflowOutflowRate, reportInflowRate,
+ reportOutflowRate);
+ this.timer.scheduleAtFixedRate(logInflowOutflowRateTask, 0, LOG_INPUT_OUTPUT_RATE_FREQUENCY);
+ this.inflowReportSenderId = metricCollector.createReportSender(connectionId, runtimeId,
+ ValueType.INFLOW_RATE, MetricType.RATE);
+ this.outflowReportSenderId = metricCollector.createReportSender(connectionId, runtimeId,
+ ValueType.OUTFLOW_RATE, MetricType.RATE);
+ }
+ }
+
+ protected void deinitializeMonitoring() {
+ if (monitorInputQueueLengthTask != null) {
+ monitorInputQueueLengthTask.cancel();
+ }
+ if (processingRateTask != null) {
+ processingRateTask.cancel();
+ }
+ if (logInflowOutflowRate || reportInflowRate || reportOutflowRate) {
+ metricCollector.removeReportSender(inflowReportSenderId);
+ metricCollector.removeReportSender(outflowReportSenderId);
+ logInflowOutflowRateTask.cancel();
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Disabled monitoring for " + this.runtimeId);
+ }
+ }
+
+ protected void postProcessFrame(long startTime, ByteBuffer frame) throws Exception {
+ if (monitorProcessingRate) {
+ frameCount++;
+ if (frameCount % PROCESS_RATE_REFRESH == 0) {
+ long endTime = System.currentTimeMillis();
+ processingRate = (int) ((double) outflowFta.getTupleCount() * 1000 / (endTime - startTime));
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Processing Rate :" + processingRate + " tuples/sec");
+ }
+ frameCount = 0;
+ }
+ }
+
+ if (logInflowOutflowRate || reportOutflowRate) {
+ metricCollector.sendReport(outflowReportSenderId, outflowFta.getTupleCount());
+ }
+
+ postProcessFrame(frame);
+
+ }
+
+ protected void preProcessFrame(ByteBuffer frame) throws Exception {
+ if (postProcessor == null) {
+ preProcessor = getFramePreProcessor();
+ }
+ if (preProcessor != null) {
+ preProcessor.preProcess(frame);
+ }
+ }
+
+ protected void postProcessFrame(ByteBuffer frame) throws Exception {
+ if (postProcessor == null) {
+ postProcessor = getFramePostProcessor();
+ }
+ if (postProcessor != null) {
+ outflowFta.reset(frame);
+ postProcessor.postProcessFrame(frame, outflowFta);
+ }
+ }
+
+ @Override
+ public void sendMessage(DataBucket message) {
+ inbox.add(message);
+ }
+
+ public void sendReport(ByteBuffer frame) {
+ if ((logInflowOutflowRate || reportInflowRate)
+ && !(inputHandler.getMode().equals(Mode.PROCESS_BACKLOG) || inputHandler.getMode().equals(
+ Mode.PROCESS_SPILL))) {
+ inflowFta.reset(frame);
+ metricCollector.sendReport(inflowReportSenderId, inflowFta.getTupleCount());
+ }
+ }
+
+ /** return rate in terms of tuples/sec **/
+ public int getInflowRate() {
+ return metricCollector.getMetric(inflowReportSenderId);
+ }
+
+ /** return rate in terms of tuples/sec **/
+ public int getOutflowRate() {
+ return metricCollector.getMetric(outflowReportSenderId);
+ }
+
+ /** return the number of pending frames from the input queue **/
+ public int getWorkSize() {
+ return inbox.size();
+ }
+
+ /** reset the number of partitions (cardinality) for the runtime **/
+ public void setNumberOfPartitions(int nPartitions) {
+ if (processingRateTask != null) {
+ int currentPartitions = ((MonitoreProcessRateTimerTask) processingRateTask).getNumberOfPartitions();
+ if (currentPartitions != nPartitions) {
+ ((MonitoreProcessRateTimerTask) processingRateTask).setNumberOfPartitions(nPartitions);
+ }
+ }
+ }
+
+ public FeedRuntimeInputHandler getInputHandler() {
+ return inputHandler;
+ }
+
+ public synchronized void close(boolean processPending, boolean disableMonitoring) {
+ super.close(processPending);
+ if (disableMonitoring) {
+ deinitializeMonitoring();
+ }
+ active = false;
+ }
+
+ @Override
+ public synchronized void processMessage(DataBucket message) throws Exception {
+ if (!active) {
+ message.doneReading();
+ return;
+ }
+ switch (message.getContentType()) {
+ case DATA:
+ boolean finishedProcessing = false;
+ ByteBuffer frameReceived = message.getContent();
+ ByteBuffer frameToProcess = null;
+ if (inputHandler.isThrottlingEnabled()) {
+ inflowFta.reset(frameReceived);
+ int pRate = getProcessingRate();
+ int inflowRate = getInflowRate();
+ if (inflowRate > pRate) {
+ double retainFraction = (pRate * 0.8 / inflowRate);
+ frameToProcess = throttleFrame(inflowFta, retainFraction);
+ inflowFta.reset(frameToProcess);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Throttling at fraction " + retainFraction + "inflow rate " + inflowRate
+ + " no of tuples remaining " + inflowFta.getTupleCount());
+
+ }
+ } else {
+ frameToProcess = frameReceived;
+ }
+ } else {
+ frameToProcess = frameReceived;
+ }
+ outflowFta.reset(frameToProcess);
+ long startTime = 0;
+ while (!finishedProcessing) {
+ try {
+ inflowFta.reset(frameToProcess);
+ startTime = System.currentTimeMillis();
+ preProcessFrame(frameToProcess);
+ frameWriter.nextFrame(frameToProcess);
+ postProcessFrame(startTime, frameToProcess);
+ finishedProcessing = true;
+ } catch (Exception e) {
+ e.printStackTrace();
+ frameToProcess = exceptionHandler.handleException(e, frameToProcess);
+ finishedProcessing = true;
+ }
+ }
+ message.doneReading();
+ break;
+ case EOD:
+ message.doneReading();
+ timer.cancel();
+ callback.frameEvent(FrameEvent.FINISHED_PROCESSING);
+ break;
+ case EOSD:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Done processing spillage");
+ }
+ message.doneReading();
+ callback.frameEvent(FrameEvent.FINISHED_PROCESSING_SPILLAGE);
+ break;
+
+ }
+ }
+
+ private ByteBuffer throttleFrame(FrameTupleAccessor fta, double retainFraction) throws HyracksDataException {
+ int desiredTuples = (int) (fta.getTupleCount() * retainFraction);
+ return FeedFrameUtil.getSampledFrame(ctx, fta, desiredTuples);
+ }
+
+ public Mode getMode() {
+ return inputHandler.getMode();
+ }
+
+ public FeedRuntimeId getRuntimeId() {
+ return runtimeId;
+ }
+
+ public void setFrameWriter(IFrameWriter frameWriter) {
+ this.frameWriter = frameWriter;
+ }
+
+ public void reset() {
+ active = true;
+ if (logInflowOutflowRate) {
+ metricCollector.resetReportSender(inflowReportSenderId);
+ metricCollector.resetReportSender(outflowReportSenderId);
+ }
+ }
+
+ public int getProcessingRate() {
+ return processingRate;
+ }
+
+ public Map<Integer, Long> getTupleTimeStats() {
+ return tupleTimeStats;
+ }
+
+ public long getAvgDelayRecordPersistence() {
+ return avgDelayPersistence;
+ }
+
+ public MonitoredBufferStorageTimerTask getStorageTimeTrackingRateTask() {
+ return storageTimeTrackingRateTask;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MonitoredBufferTimerTasks.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MonitoredBufferTimerTasks.java
new file mode 100644
index 0000000..13c979f
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/MonitoredBufferTimerTasks.java
@@ -0,0 +1,290 @@
+/*
+ * 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.common.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.Set;
+import java.util.TimerTask;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.config.AsterixFeedProperties;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessageService;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector.ValueType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.Mode;
+import edu.uci.ics.asterix.common.feeds.api.IFrameEventCallback;
+import edu.uci.ics.asterix.common.feeds.api.IFrameEventCallback.FrameEvent;
+import edu.uci.ics.asterix.common.feeds.message.FeedReportMessage;
+import edu.uci.ics.asterix.common.feeds.message.ScaleInReportMessage;
+import edu.uci.ics.asterix.common.feeds.message.StorageReportFeedMessage;
+
+public class MonitoredBufferTimerTasks {
+
+ private static final Logger LOGGER = Logger.getLogger(MonitorInputQueueLengthTimerTask.class.getName());
+
+ public static class MonitoredBufferStorageTimerTask extends TimerTask {
+
+ private static final int PERSISTENCE_DELAY_VIOLATION_MAX = 5;
+
+ private final StorageSideMonitoredBuffer mBuffer;
+ private final IFeedManager feedManager;
+ private final int partition;
+ private final FeedConnectionId connectionId;
+ private final FeedPolicyAccessor policyAccessor;
+ private final StorageFrameHandler storageFromeHandler;
+ private final StorageReportFeedMessage storageReportMessage;
+ private final FeedTupleCommitAckMessage tupleCommitAckMessage;
+
+ private Map<Integer, Integer> maxIntakeBaseCovered;
+ private int countDelayExceeded = 0;
+
+ public MonitoredBufferStorageTimerTask(StorageSideMonitoredBuffer mBuffer, IFeedManager feedManager,
+ FeedConnectionId connectionId, int partition, FeedPolicyAccessor policyAccessor,
+ StorageFrameHandler storageFromeHandler) {
+ this.mBuffer = mBuffer;
+ this.feedManager = feedManager;
+ this.connectionId = connectionId;
+ this.partition = partition;
+ this.policyAccessor = policyAccessor;
+ this.storageFromeHandler = storageFromeHandler;
+ this.storageReportMessage = new StorageReportFeedMessage(this.connectionId, this.partition, 0, false, 0, 0);
+ this.tupleCommitAckMessage = new FeedTupleCommitAckMessage(this.connectionId, 0, 0, null);
+ this.maxIntakeBaseCovered = new HashMap<Integer, Integer>();
+ }
+
+ @Override
+ public void run() {
+ if (mBuffer.isAckingEnabled() && !mBuffer.getInputHandler().isThrottlingEnabled()) {
+ ackRecords();
+ }
+ if (mBuffer.isTimeTrackingEnabled()) {
+ checkLatencyViolation();
+ }
+ }
+
+ private void ackRecords() {
+ Set<Integer> partitions = storageFromeHandler.getPartitionsWithStats();
+ List<Integer> basesCovered = new ArrayList<Integer>();
+ for (int intakePartition : partitions) {
+ Map<Integer, IntakePartitionStatistics> baseAcks = storageFromeHandler
+ .getBaseAcksForPartition(intakePartition);
+ for (Entry<Integer, IntakePartitionStatistics> entry : baseAcks.entrySet()) {
+ int base = entry.getKey();
+ IntakePartitionStatistics stats = entry.getValue();
+ Integer maxIntakeBaseForPartition = maxIntakeBaseCovered.get(intakePartition);
+ if (maxIntakeBaseForPartition == null || maxIntakeBaseForPartition < base) {
+ tupleCommitAckMessage.reset(intakePartition, base, stats.getAckInfo());
+ feedManager.getFeedMessageService().sendMessage(tupleCommitAckMessage);
+ } else {
+ basesCovered.add(base);
+ }
+ }
+ for (Integer b : basesCovered) {
+ baseAcks.remove(b);
+ }
+ basesCovered.clear();
+ }
+ }
+
+ private void checkLatencyViolation() {
+ long avgDelayPersistence = storageFromeHandler.getAvgDelayPersistence();
+ if (avgDelayPersistence > policyAccessor.getMaxDelayRecordPersistence()) {
+ countDelayExceeded++;
+ if (countDelayExceeded > PERSISTENCE_DELAY_VIOLATION_MAX) {
+ storageReportMessage.reset(0, false, mBuffer.getAvgDelayRecordPersistence());
+ feedManager.getFeedMessageService().sendMessage(storageReportMessage);
+ }
+ } else {
+ countDelayExceeded = 0;
+ }
+ }
+
+ public void receiveCommitAckResponse(FeedTupleCommitResponseMessage message) {
+ maxIntakeBaseCovered.put(message.getIntakePartition(), message.getMaxWindowAcked());
+ }
+ }
+
+ public static class LogInputOutputRateTask extends TimerTask {
+
+ private final MonitoredBuffer mBuffer;
+ private final boolean log;
+ private final boolean reportInflow;
+ private final boolean reportOutflow;
+
+ private final IFeedMessageService messageService;
+ private final FeedReportMessage message;
+
+ public LogInputOutputRateTask(MonitoredBuffer mBuffer, boolean log, boolean reportInflow, boolean reportOutflow) {
+ this.mBuffer = mBuffer;
+ this.log = log;
+ this.reportInflow = reportInflow;
+ this.reportOutflow = reportOutflow;
+ if (reportInflow || reportOutflow) {
+ ValueType vType = reportInflow ? ValueType.INFLOW_RATE : ValueType.OUTFLOW_RATE;
+ messageService = mBuffer.getInputHandler().getFeedManager().getFeedMessageService();
+ message = new FeedReportMessage(mBuffer.getInputHandler().getConnectionId(), mBuffer.getRuntimeId(),
+ vType, 0);
+ } else {
+ messageService = null;
+ message = null;
+ }
+
+ }
+
+ @Override
+ public void run() {
+ int pendingWork = mBuffer.getWorkSize();
+ int outflowRate = mBuffer.getOutflowRate();
+ int inflowRate = mBuffer.getInflowRate();
+ if (log) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(mBuffer.getRuntimeId() + " " + "Inflow rate:" + inflowRate + " Outflow Rate:"
+ + outflowRate + " Pending Work " + pendingWork);
+ }
+ }
+ if (reportInflow) {
+ message.reset(inflowRate);
+ } else if (reportOutflow) {
+ message.reset(outflowRate);
+ }
+ messageService.sendMessage(message);
+ }
+ }
+
+ public static class MonitorInputQueueLengthTimerTask extends TimerTask {
+
+ private final MonitoredBuffer mBuffer;
+ private final IFrameEventCallback callback;
+ private final int pendingWorkThreshold;
+ private final int maxSuccessiveThresholdPeriods;
+ private FrameEvent lastEvent = FrameEvent.NO_OP;
+ private int pendingWorkExceedCount = 0;
+
+ public MonitorInputQueueLengthTimerTask(MonitoredBuffer mBuffer, IFrameEventCallback callback) {
+ this.mBuffer = mBuffer;
+ this.callback = callback;
+ AsterixFeedProperties props = mBuffer.getInputHandler().getFeedManager().getAsterixFeedProperties();
+ pendingWorkThreshold = props.getPendingWorkThreshold();
+ maxSuccessiveThresholdPeriods = props.getMaxSuccessiveThresholdPeriod();
+ }
+
+ @Override
+ public void run() {
+ int pendingWork = mBuffer.getWorkSize();
+ if (mBuffer.getMode().equals(Mode.PROCESS_SPILL) || mBuffer.getMode().equals(Mode.PROCESS_BACKLOG)) {
+ return;
+ }
+
+ switch (lastEvent) {
+ case NO_OP:
+ case PENDING_WORK_DONE:
+ case FINISHED_PROCESSING_SPILLAGE:
+ if (pendingWork > pendingWorkThreshold) {
+ pendingWorkExceedCount++;
+ if (pendingWorkExceedCount > maxSuccessiveThresholdPeriods) {
+ pendingWorkExceedCount = 0;
+ lastEvent = FrameEvent.PENDING_WORK_THRESHOLD_REACHED;
+ callback.frameEvent(lastEvent);
+ }
+ } else if (pendingWork == 0 && mBuffer.getMode().equals(Mode.SPILL)) {
+ lastEvent = FrameEvent.PENDING_WORK_DONE;
+ callback.frameEvent(lastEvent);
+ }
+ break;
+ case PENDING_WORK_THRESHOLD_REACHED:
+ if (((pendingWork * 1.0) / pendingWorkThreshold) <= 0.5) {
+ lastEvent = FrameEvent.PENDING_WORK_DONE;
+ callback.frameEvent(lastEvent);
+ }
+ break;
+ case FINISHED_PROCESSING:
+ break;
+
+ }
+ }
+ }
+
+ /**
+ * A timer task to measure and compare the processing rate and inflow rate
+ * to look for possibility to scale-in, that is reduce the degree of cardinality
+ * of the compute operator.
+ */
+ public static class MonitoreProcessRateTimerTask extends TimerTask {
+
+ private final MonitoredBuffer mBuffer;
+ private final IFeedManager feedManager;
+ private int nPartitions;
+ private ScaleInReportMessage sMessage;
+ private boolean proposedChange;
+
+ public MonitoreProcessRateTimerTask(MonitoredBuffer mBuffer, IFeedManager feedManager,
+ FeedConnectionId connectionId, int nPartitions) {
+ this.mBuffer = mBuffer;
+ this.feedManager = feedManager;
+ this.nPartitions = nPartitions;
+ this.sMessage = new ScaleInReportMessage(connectionId, FeedRuntimeType.COMPUTE, 0, 0);
+ this.proposedChange = false;
+ }
+
+ public int getNumberOfPartitions() {
+ return nPartitions;
+ }
+
+ public void setNumberOfPartitions(int nPartitions) {
+ this.nPartitions = nPartitions;
+ proposedChange = false;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Reset the number of partitions for " + mBuffer.getRuntimeId() + " to " + nPartitions);
+ }
+ }
+
+ @Override
+ public void run() {
+ if (!proposedChange) {
+ int inflowRate = mBuffer.getInflowRate();
+ int procRate = mBuffer.getProcessingRate();
+ if (inflowRate > 0 && procRate > 0) {
+ if (inflowRate < procRate) {
+ int possibleCardinality = (int) Math.ceil(nPartitions * inflowRate / (double) procRate);
+ if (possibleCardinality < nPartitions
+ && ((((nPartitions - possibleCardinality) * 1.0) / nPartitions) >= 0.25)) {
+ sMessage.reset(nPartitions, possibleCardinality);
+ feedManager.getFeedMessageService().sendMessage(sMessage);
+ proposedChange = true;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Proposed scale-in " + sMessage);
+ }
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Inflow Rate (" + inflowRate + ") exceeds Processing Rate" + " (" + procRate
+ + ")");
+ }
+ }
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Waiting for earlier proposal to scale in to be applied");
+ }
+ }
+ }
+ }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/NodeLoad.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/NodeLoad.java
new file mode 100644
index 0000000..66e5a40
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/NodeLoad.java
@@ -0,0 +1,44 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+
+public class NodeLoad implements Comparable<NodeLoad> {
+
+ private final String nodeId;
+
+ private int nRuntimes;
+
+ public NodeLoad(String nodeId) {
+ this.nodeId = nodeId;
+ this.nRuntimes = 0;
+ }
+
+ public void addLoad() {
+ nRuntimes++;
+ }
+
+ public void removeLoad(FeedRuntimeType runtimeType) {
+ nRuntimes--;
+ }
+
+ @Override
+ public int compareTo(NodeLoad o) {
+ if (this == o) {
+ return 0;
+ }
+ return nRuntimes - o.getnRuntimes();
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public int getnRuntimes() {
+ return nRuntimes;
+ }
+
+ public void setnRuntimes(int nRuntimes) {
+ this.nRuntimes = nRuntimes;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/NodeLoadReport.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/NodeLoadReport.java
new file mode 100644
index 0000000..8257143
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/NodeLoadReport.java
@@ -0,0 +1,95 @@
+/*
+ * Copyright 2009-2014 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.common.feeds;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+public class NodeLoadReport implements Comparable<NodeLoadReport> {
+
+ private final String nodeId;
+ private float cpuLoad;
+ private double usedHeap;
+ private int nRuntimes;
+
+ public NodeLoadReport(String nodeId, float cpuLoad, float usedHeap, int nRuntimes) {
+ this.nodeId = nodeId;
+ this.cpuLoad = cpuLoad;
+ this.usedHeap = usedHeap;
+ this.nRuntimes = nRuntimes;
+ }
+
+ public static NodeLoadReport read(JSONObject obj) throws JSONException {
+ NodeLoadReport r = new NodeLoadReport(obj.getString(FeedConstants.MessageConstants.NODE_ID),
+ (float) obj.getDouble(FeedConstants.MessageConstants.CPU_LOAD),
+ (float) obj.getDouble(FeedConstants.MessageConstants.HEAP_USAGE),
+ obj.getInt(FeedConstants.MessageConstants.N_RUNTIMES));
+ return r;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof NodeLoadReport)) {
+ return false;
+ }
+ return ((NodeLoadReport) o).nodeId.equals(nodeId);
+ }
+
+ @Override
+ public int hashCode() {
+ return nodeId.hashCode();
+ }
+
+ @Override
+ public int compareTo(NodeLoadReport o) {
+ if (nRuntimes != o.getnRuntimes()) {
+ return nRuntimes - o.getnRuntimes();
+ } else {
+ return (int) (this.cpuLoad - ((NodeLoadReport) o).cpuLoad);
+ }
+ }
+
+ public float getCpuLoad() {
+ return cpuLoad;
+ }
+
+ public void setCpuLoad(float cpuLoad) {
+ this.cpuLoad = cpuLoad;
+ }
+
+ public double getUsedHeap() {
+ return usedHeap;
+ }
+
+ public void setUsedHeap(double usedHeap) {
+ this.usedHeap = usedHeap;
+ }
+
+ public int getnRuntimes() {
+ return nRuntimes;
+ }
+
+ public void setnRuntimes(int nRuntimes) {
+ this.nRuntimes = nRuntimes;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/NodeLoadReportService.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/NodeLoadReportService.java
new file mode 100644
index 0000000..383f869
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/NodeLoadReportService.java
@@ -0,0 +1,104 @@
+/*
+ * 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.common.feeds;
+
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+import java.lang.management.OperatingSystemMXBean;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessageService;
+import edu.uci.ics.asterix.common.feeds.api.IFeedService;
+import edu.uci.ics.asterix.common.feeds.message.NodeReportMessage;
+
+public class NodeLoadReportService implements IFeedService {
+
+ private static final int NODE_LOAD_REPORT_FREQUENCY = 2000;
+ private static final float CPU_CHANGE_THRESHOLD = 0.2f;
+ private static final float HEAP_CHANGE_THRESHOLD = 0.4f;
+
+ private final String nodeId;
+ private final NodeLoadReportTask task;
+ private final Timer timer;
+
+ public NodeLoadReportService(String nodeId, IFeedManager feedManager) {
+ this.nodeId = nodeId;
+ this.task = new NodeLoadReportTask(nodeId, feedManager);
+ this.timer = new Timer();
+ }
+
+ @Override
+ public void start() throws Exception {
+ timer.schedule(task, 0, NODE_LOAD_REPORT_FREQUENCY);
+ }
+
+ @Override
+ public void stop() {
+ timer.cancel();
+ }
+
+ private static class NodeLoadReportTask extends TimerTask {
+
+ private final String nodeId;
+ private final IFeedManager feedManager;
+ private final NodeReportMessage message;
+ private final IFeedMessageService messageService;
+
+ private static OperatingSystemMXBean osBean = ManagementFactory.getOperatingSystemMXBean();
+ private static MemoryMXBean memBean = ManagementFactory.getMemoryMXBean();
+
+ public NodeLoadReportTask(String nodeId, IFeedManager feedManager) {
+ this.nodeId = nodeId;
+ this.feedManager = feedManager;
+ this.message = new NodeReportMessage(0.0f, 0L, 0);
+ this.messageService = feedManager.getFeedMessageService();
+ }
+
+ @Override
+ public void run() {
+ List<FeedRuntimeId> runtimeIds = feedManager.getFeedConnectionManager().getRegisteredRuntimes();
+ int nRuntimes = runtimeIds.size();
+ double cpuLoad = getCpuLoad();
+ double usedHeap = getUsedHeap();
+ if (sendMessage(nRuntimes, cpuLoad, usedHeap)) {
+ message.reset(cpuLoad, usedHeap, nRuntimes);
+ messageService.sendMessage(message);
+ }
+ }
+
+ private boolean sendMessage(int nRuntimes, double cpuLoad, double usedHeap) {
+ if (message == null) {
+ return true;
+ }
+
+ boolean changeInCpu = (Math.abs(cpuLoad - message.getCpuLoad()) / message.getCpuLoad()) > CPU_CHANGE_THRESHOLD;
+ boolean changeInUsedHeap = (Math.abs(usedHeap - message.getUsedHeap()) / message.getUsedHeap()) > HEAP_CHANGE_THRESHOLD;
+ boolean changeInRuntimeSize = nRuntimes != message.getnRuntimes();
+ return changeInCpu || changeInUsedHeap || changeInRuntimeSize;
+ }
+
+ private double getCpuLoad() {
+ return osBean.getSystemLoadAverage();
+ }
+
+ private double getUsedHeap() {
+ return ((double) memBean.getHeapMemoryUsage().getUsed()) / memBean.getHeapMemoryUsage().getMax();
+ }
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/Series.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/Series.java
new file mode 100644
index 0000000..780ea03
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/Series.java
@@ -0,0 +1,26 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector.MetricType;
+
+public abstract class Series {
+
+ protected final MetricType type;
+ protected int runningSum;
+
+ public Series(MetricType type) {
+ this.type = type;
+ }
+
+ public abstract void addValue(int value);
+
+ public int getRunningSum() {
+ return runningSum;
+ }
+
+ public MetricType getType() {
+ return type;
+ }
+
+ public abstract void reset();
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SeriesAvg.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SeriesAvg.java
new file mode 100644
index 0000000..f9e33d6
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SeriesAvg.java
@@ -0,0 +1,29 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector.MetricType;
+
+public class SeriesAvg extends Series {
+
+ private int count;
+
+ public SeriesAvg() {
+ super(MetricType.AVG);
+ }
+
+ public int getAvg() {
+ return runningSum / count;
+ }
+
+ public synchronized void addValue(int value) {
+ if (value < 0) {
+ return;
+ }
+ runningSum += value;
+ count++;
+ }
+
+ public void reset(){
+ count = 0;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SeriesRate.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SeriesRate.java
new file mode 100644
index 0000000..8271462
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SeriesRate.java
@@ -0,0 +1,74 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.util.Timer;
+import java.util.TimerTask;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector.MetricType;
+
+public class SeriesRate extends Series {
+
+ private static final long REFRESH_MEASUREMENT = 5000; // 5 seconds
+
+ private int rate;
+ private Timer timer;
+ private RateComputingTask task;
+
+ public SeriesRate() {
+ super(MetricType.RATE);
+ begin();
+ }
+
+ public int getRate() {
+ return rate;
+ }
+
+ public synchronized void addValue(int value) {
+ if (value < 0) {
+ return;
+ }
+ runningSum += value;
+ }
+
+ public void begin() {
+ if (timer == null) {
+ timer = new Timer();
+ task = new RateComputingTask(this);
+ timer.scheduleAtFixedRate(task, 0, REFRESH_MEASUREMENT);
+ }
+ }
+
+ public void end() {
+ if (timer != null) {
+ timer.cancel();
+ }
+ }
+
+ public void reset() {
+ rate = 0;
+ if (task != null) {
+ task.reset();
+ }
+ }
+
+ private class RateComputingTask extends TimerTask {
+
+ private int lastMeasured = 0;
+ private final SeriesRate series;
+
+ public RateComputingTask(SeriesRate series) {
+ this.series = series;
+ }
+
+ @Override
+ public void run() {
+ int currentValue = series.getRunningSum();
+ rate = (int) (((currentValue - lastMeasured) * 1000) / REFRESH_MEASUREMENT);
+ lastMeasured = currentValue;
+ }
+
+ public void reset() {
+ lastMeasured = 0;
+ }
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/StorageFrameHandler.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/StorageFrameHandler.java
new file mode 100644
index 0000000..e3348cf
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/StorageFrameHandler.java
@@ -0,0 +1,100 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import edu.uci.ics.asterix.common.feeds.FeedConstants.StatisticsConstants;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class StorageFrameHandler {
+
+ private final Map<Integer, Map<Integer, IntakePartitionStatistics>> intakeStatistics;
+ private long avgDelayPersistence;
+
+ public StorageFrameHandler() {
+ intakeStatistics = new HashMap<Integer, Map<Integer, IntakePartitionStatistics>>();
+ avgDelayPersistence = 0L;
+ }
+
+ public synchronized void updateTrackingInformation(ByteBuffer frame, FrameTupleAccessor frameAccessor) {
+ int nTuples = frameAccessor.getTupleCount();
+ long delay = 0;
+ long intakeTimestamp;
+ long currentTime = System.currentTimeMillis();
+ int partition = 0;
+ int recordId = 0;
+ for (int i = 0; i < nTuples; i++) {
+ int recordStart = frameAccessor.getTupleStartOffset(i) + frameAccessor.getFieldSlotsLength();
+ int openPartOffsetOrig = frame.getInt(recordStart + 6);
+ int numOpenFields = frame.getInt(recordStart + openPartOffsetOrig);
+
+ int recordIdOffset = openPartOffsetOrig + 4 + 8 * numOpenFields
+ + (StatisticsConstants.INTAKE_TUPLEID.length() + 2) + 1;
+ recordId = frame.getInt(recordStart + recordIdOffset);
+
+ int partitionOffset = recordIdOffset + 4 + (StatisticsConstants.INTAKE_PARTITION.length() + 2) + 1;
+ partition = frame.getInt(recordStart + partitionOffset);
+
+ ackRecordId(partition, recordId);
+ int intakeTimestampValueOffset = partitionOffset + 4 + (StatisticsConstants.INTAKE_TIMESTAMP.length() + 2)
+ + 1;
+ intakeTimestamp = frame.getLong(recordStart + intakeTimestampValueOffset);
+
+ int storeTimestampValueOffset = intakeTimestampValueOffset + 8
+ + (StatisticsConstants.STORE_TIMESTAMP.length() + 2) + 1;
+ frame.putLong(recordStart + storeTimestampValueOffset, System.currentTimeMillis());
+ delay += currentTime - intakeTimestamp;
+ }
+ avgDelayPersistence = delay / nTuples;
+ }
+
+ private void ackRecordId(int partition, int recordId) {
+ Map<Integer, IntakePartitionStatistics> map = intakeStatistics.get(partition);
+ if (map == null) {
+ map = new HashMap<Integer, IntakePartitionStatistics>();
+ intakeStatistics.put(partition, map);
+ }
+ int base = (int) Math.ceil(recordId * 1.0 / IntakePartitionStatistics.ACK_WINDOW_SIZE);
+ IntakePartitionStatistics intakeStatsForBaseOfPartition = map.get(base);
+ if (intakeStatsForBaseOfPartition == null) {
+ intakeStatsForBaseOfPartition = new IntakePartitionStatistics(partition, base);
+ map.put(base, intakeStatsForBaseOfPartition);
+ }
+ intakeStatsForBaseOfPartition.ackRecordId(recordId);
+ }
+
+ public byte[] getAckData(int partition, int base) {
+ Map<Integer, IntakePartitionStatistics> intakeStats = intakeStatistics.get(partition);
+ if (intakeStats != null) {
+ IntakePartitionStatistics intakePartitionStats = intakeStats.get(base);
+ if (intakePartitionStats != null) {
+ return intakePartitionStats.getAckInfo();
+ }
+ }
+ return null;
+ }
+
+ public synchronized Map<Integer, IntakePartitionStatistics> getBaseAcksForPartition(int partition) {
+ Map<Integer, IntakePartitionStatistics> intakeStatsForPartition = intakeStatistics.get(partition);
+ Map<Integer, IntakePartitionStatistics> clone = new HashMap<Integer, IntakePartitionStatistics>();
+ for (Entry<Integer, IntakePartitionStatistics> entry : intakeStatsForPartition.entrySet()) {
+ clone.put(entry.getKey(), entry.getValue());
+ }
+ return intakeStatsForPartition;
+ }
+
+ public long getAvgDelayPersistence() {
+ return avgDelayPersistence;
+ }
+
+ public void setAvgDelayPersistence(long avgDelayPersistence) {
+ this.avgDelayPersistence = avgDelayPersistence;
+ }
+
+ public Set<Integer> getPartitionsWithStats() {
+ return intakeStatistics.keySet();
+ }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/StorageSideMonitoredBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/StorageSideMonitoredBuffer.java
new file mode 100644
index 0000000..947fb32
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/StorageSideMonitoredBuffer.java
@@ -0,0 +1,189 @@
+package edu.uci.ics.asterix.common.feeds;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.common.feeds.FeedConstants.StatisticsConstants;
+import edu.uci.ics.asterix.common.feeds.api.IExceptionHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector;
+import edu.uci.ics.asterix.common.feeds.api.IFrameEventCallback;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class StorageSideMonitoredBuffer extends MonitoredBuffer {
+
+ private static final long STORAGE_TIME_TRACKING_FREQUENCY = 5000; // 10
+ // seconds
+
+ private boolean ackingEnabled;
+ private final boolean timeTrackingEnabled;
+
+ public StorageSideMonitoredBuffer(IHyracksTaskContext ctx, FeedRuntimeInputHandler inputHandler, IFrameWriter frameWriter,
+ FrameTupleAccessor fta, RecordDescriptor recordDesc, IFeedMetricCollector metricCollector,
+ FeedConnectionId connectionId, FeedRuntimeId runtimeId, IExceptionHandler exceptionHandler,
+ IFrameEventCallback callback, int nPartitions, FeedPolicyAccessor policyAccessor) {
+ super(ctx, inputHandler, frameWriter, fta, recordDesc, metricCollector, connectionId, runtimeId,
+ exceptionHandler, callback, nPartitions, policyAccessor);
+ timeTrackingEnabled = policyAccessor.isTimeTrackingEnabled();
+ ackingEnabled = policyAccessor.atleastOnceSemantics();
+ if (ackingEnabled || timeTrackingEnabled) {
+ storageFromeHandler = new StorageFrameHandler();
+ this.storageTimeTrackingRateTask = new MonitoredBufferTimerTasks.MonitoredBufferStorageTimerTask(this,
+ inputHandler.getFeedManager(), connectionId, runtimeId.getPartition(), policyAccessor,
+ storageFromeHandler);
+ this.timer.scheduleAtFixedRate(storageTimeTrackingRateTask, 0, STORAGE_TIME_TRACKING_FREQUENCY);
+ }
+ }
+
+ @Override
+ protected boolean monitorProcessingRate() {
+ return false;
+ }
+
+ protected boolean logInflowOutflowRate() {
+ return true;
+ }
+
+ @Override
+ public IFramePreprocessor getFramePreProcessor() {
+ return new IFramePreprocessor() {
+
+ @Override
+ public void preProcess(ByteBuffer frame) {
+ try {
+ if (ackingEnabled) {
+ storageFromeHandler.updateTrackingInformation(frame, inflowFta);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ };
+ }
+
+ @Override
+ protected IFramePostProcessor getFramePostProcessor() {
+ return new IFramePostProcessor() {
+
+ private static final long NORMAL_WINDOW_LIMIT = 400 * 1000;
+ private static final long HIGH_WINDOW_LIMIT = 800 * 1000;
+ private static final long LOW_WINDOW_LIMIT = 1200 * 1000;
+
+ private long delayNormalWindow = 0;
+ private long delayHighWindow = 0;
+ private long delayLowWindow = 0;
+
+ private int countNormalWindow;
+ private int countHighWindow;
+ private int countLowWindow;
+
+ private long beginIntakeTimestamp = 0;
+
+ @Override
+ public void postProcessFrame(ByteBuffer frame, FrameTupleAccessor frameAccessor) {
+ if (ackingEnabled || timeTrackingEnabled) {
+ int nTuples = frameAccessor.getTupleCount();
+ long intakeTimestamp;
+ long currentTime = System.currentTimeMillis();
+ int partition = 0;
+ int recordId = 0;
+ for (int i = 0; i < nTuples; i++) {
+ int recordStart = frameAccessor.getTupleStartOffset(i) + frameAccessor.getFieldSlotsLength();
+ int openPartOffsetOrig = frame.getInt(recordStart + 6);
+ int numOpenFields = frame.getInt(recordStart + openPartOffsetOrig);
+
+ int recordIdOffset = openPartOffsetOrig + 4 + 8 * numOpenFields
+ + (StatisticsConstants.INTAKE_TUPLEID.length() + 2) + 1;
+ recordId = frame.getInt(recordStart + recordIdOffset);
+
+ int partitionOffset = recordIdOffset + 4 + (StatisticsConstants.INTAKE_PARTITION.length() + 2)
+ + 1;
+ partition = frame.getInt(recordStart + partitionOffset);
+
+ int intakeTimestampValueOffset = partitionOffset + 4
+ + (StatisticsConstants.INTAKE_TIMESTAMP.length() + 2) + 1;
+ intakeTimestamp = frame.getLong(recordStart + intakeTimestampValueOffset);
+ if (beginIntakeTimestamp == 0) {
+ beginIntakeTimestamp = intakeTimestamp;
+ LOGGER.warning("Begin Timestamp: " + beginIntakeTimestamp);
+ }
+
+ updateRunningAvg(intakeTimestamp, currentTime);
+
+ int storeTimestampValueOffset = intakeTimestampValueOffset + 8
+ + (StatisticsConstants.STORE_TIMESTAMP.length() + 2) + 1;
+ frame.putLong(recordStart + storeTimestampValueOffset, System.currentTimeMillis());
+ }
+ logRunningAvg();
+ resetRunningAvg();
+ }
+ }
+
+ private void updateRunningAvg(long intakeTimestamp, long currentTime) {
+ long diffTimestamp = intakeTimestamp - beginIntakeTimestamp;
+ long delay = (currentTime - intakeTimestamp);
+ if (diffTimestamp < NORMAL_WINDOW_LIMIT) {
+ delayNormalWindow += delay;
+ countNormalWindow++;
+ } else if (diffTimestamp < HIGH_WINDOW_LIMIT) {
+ delayHighWindow += delay;
+ countHighWindow++;
+ } else {
+ delayLowWindow += delay;
+ countLowWindow++;
+ }
+ }
+
+ private void resetRunningAvg() {
+ delayNormalWindow = 0;
+ countNormalWindow = 0;
+ delayHighWindow = 0;
+ countHighWindow = 0;
+ delayLowWindow = 0;
+ countLowWindow = 0;
+ }
+
+ private void logRunningAvg() {
+ if (countNormalWindow != 0 && delayNormalWindow != 0) {
+ LOGGER.warning("Window:" + 0 + ":" + "Avg Travel_Time:" + (delayNormalWindow / countNormalWindow));
+ }
+ if (countHighWindow != 0 && delayHighWindow != 0) {
+ LOGGER.warning("Window:" + 1 + ":" + "Avg Travel_Time:" + (delayHighWindow / countHighWindow));
+ }
+ if (countLowWindow != 0 && delayLowWindow != 0) {
+ LOGGER.warning("Window:" + 2 + ":" + "Avg Travel_Time:" + (delayLowWindow / countLowWindow));
+ }
+ }
+
+ };
+ }
+
+ public boolean isAckingEnabled() {
+ return ackingEnabled;
+ }
+
+ public void setAcking(boolean ackingEnabled) {
+ this.ackingEnabled = ackingEnabled;
+ }
+
+ public boolean isTimeTrackingEnabled() {
+ return timeTrackingEnabled;
+ }
+
+ @Override
+ protected boolean monitorInputQueueLength() {
+ return true;
+ }
+
+ @Override
+ protected boolean reportOutflowRate() {
+ return true;
+ }
+
+ @Override
+ protected boolean reportInflowRate() {
+ return false;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SubscribableFeedRuntimeId.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SubscribableFeedRuntimeId.java
new file mode 100644
index 0000000..db958b3
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SubscribableFeedRuntimeId.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.common.feeds;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+
+public class SubscribableFeedRuntimeId extends FeedRuntimeId {
+
+ private final FeedId feedId;
+
+ public SubscribableFeedRuntimeId(FeedId feedId, FeedRuntimeType runtimeType, int partition) {
+ super(runtimeType, partition, FeedRuntimeId.DEFAULT_OPERAND_ID);
+ this.feedId = feedId;
+ }
+
+ public FeedId getFeedId() {
+ return feedId;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (!(o instanceof SubscribableFeedRuntimeId)) {
+ return false;
+ }
+
+ return (super.equals(o) && this.feedId.equals(((SubscribableFeedRuntimeId) o).getFeedId()));
+ }
+
+ @Override
+ public int hashCode() {
+ return super.hashCode() + feedId.hashCode();
+ }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SubscribableRuntime.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SubscribableRuntime.java
new file mode 100644
index 0000000..7fa2869
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SubscribableRuntime.java
@@ -0,0 +1,86 @@
+/*
+ * 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.common.feeds;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.feeds.api.ISubscribableRuntime;
+import edu.uci.ics.asterix.common.feeds.api.ISubscriberRuntime;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class SubscribableRuntime extends FeedRuntime implements ISubscribableRuntime {
+
+ protected static final Logger LOGGER = Logger.getLogger(SubscribableRuntime.class.getName());
+
+ protected final FeedId feedId;
+ protected final List<ISubscriberRuntime> subscribers;
+ protected final RecordDescriptor recordDescriptor;
+ protected final DistributeFeedFrameWriter dWriter;
+
+ public SubscribableRuntime(FeedId feedId, FeedRuntimeId runtimeId, FeedRuntimeInputHandler inputHandler,
+ DistributeFeedFrameWriter dWriter, RecordDescriptor recordDescriptor) {
+ super(runtimeId, inputHandler, dWriter);
+ this.feedId = feedId;
+ this.recordDescriptor = recordDescriptor;
+ this.dWriter = dWriter;
+ this.subscribers = new ArrayList<ISubscriberRuntime>();
+ }
+
+ public FeedId getFeedId() {
+ return feedId;
+ }
+
+ @Override
+ public String toString() {
+ return "SubscribableRuntime" + " [" + feedId + "]" + "(" + runtimeId + ")";
+ }
+
+ @Override
+ public synchronized void subscribeFeed(FeedPolicyAccessor fpa, CollectionRuntime collectionRuntime)
+ throws Exception {
+ FeedFrameCollector collector = dWriter.subscribeFeed(new FeedPolicyAccessor(collectionRuntime.getFeedPolicy()),
+ collectionRuntime.getInputHandler(), collectionRuntime.getConnectionId());
+ collectionRuntime.setFrameCollector(collector);
+ subscribers.add(collectionRuntime);
+ }
+
+ @Override
+ public synchronized void unsubscribeFeed(CollectionRuntime collectionRuntime) throws Exception {
+ dWriter.unsubscribeFeed(collectionRuntime.getFeedFrameWriter());
+ subscribers.remove(collectionRuntime);
+ }
+
+ @Override
+ public synchronized List<ISubscriberRuntime> getSubscribers() {
+ return subscribers;
+ }
+
+ @Override
+ public DistributeFeedFrameWriter getFeedFrameWriter() {
+ return dWriter;
+ }
+
+ public FeedRuntimeType getFeedRuntimeType() {
+ return runtimeId.getFeedRuntimeType();
+ }
+
+ @Override
+ public RecordDescriptor getRecordDescriptor() {
+ return recordDescriptor;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SuperFeedManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SuperFeedManager.java
deleted file mode 100644
index 37306a0..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/SuperFeedManager.java
+++ /dev/null
@@ -1,447 +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.common.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;
-
-/**
- * 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;
-
- private final IFeedManager feedManager;
-
- public static final int PORT_RANGE_ASSIGNED = 10;
-
- public enum FeedReportMessageType {
- CONGESTION,
- THROUGHPUT
- }
-
- public SuperFeedManager(FeedConnectionId feedId, String host, String nodeId, int port, IFeedManager feedManager)
- throws Exception {
- this.feedConnectionId = feedId;
- this.feedManager = feedManager;
- 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.getFeedExecutorService(feedConnectionId);
- sfmService = new FeedReportDestinationSocketProvider(feedReportPort, feedReportInbox, feedConnectionId,
- availablePort, feedManager);
- executorService.execute(sfmService);
- subscriptionService = new SuperFeedReportSubscriptionService(feedConnectionId, feedReportSubscribePort,
- sfmService.getMesgAnalyzer(), availablePort, feedManager);
- 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>();
- private final IFeedManager feedManager;
-
- public SuperFeedReportSubscriptionService(FeedConnectionId feedId, int port, FeedReportProvider reportProvider,
- AtomicInteger nextPort, IFeedManager feedManager) throws IOException {
- this.feedId = feedId;
- serverFeedSubscribe = feedManager.getFeedRuntimeManager(feedId).createServerSocket(port);
- this.subscriptionPort = nextPort;
- this.reportProvider = reportProvider;
- this.feedManager = feedManager;
- }
-
- 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, feedManager);
- dataProviders.add(dataProviderService);
- feedManager.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 final IFeedManager feedManager;
- 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,
- IFeedManager feedManager) throws IOException {
- this.feedId = feedId;
- this.subscriptionPort = port;
- this.inbox = inbox;
- dataProviderSocket = feedManager.getFeedRuntimeManager(feedId).createServerSocket(port);
- this.feedManager = feedManager;
- }
-
- @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, IFeedManager feedManager) throws IOException {
- FeedRuntimeManager runtimeManager = feedManager.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.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 + FeedMessageService.MessageSeparator);
- } 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-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IAdapterRuntimeManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IAdapterRuntimeManager.java
new file mode 100644
index 0000000..cb84646
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IAdapterRuntimeManager.java
@@ -0,0 +1,80 @@
+/*
+ * 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.common.feeds.api;
+
+import edu.uci.ics.asterix.common.feeds.FeedId;
+
+public interface IAdapterRuntimeManager {
+
+ public enum State {
+ /**
+ * Indicates that AsterixDB is maintaining the flow of data from external source into its storage.
+ */
+ ACTIVE_INGESTION,
+
+ /**
+ * Indicates that data from external source is being buffered and not
+ * pushed downstream
+ */
+
+ INACTIVE_INGESTION,
+ /**
+ * Indicates that feed ingestion activity has finished.
+ */
+ FINISHED_INGESTION,
+
+ /** Indicates the occurrence of a failure during the intake stage of a data ingestion pipeline **/
+ FAILED_INGESTION
+ }
+
+ /**
+ * Start feed ingestion
+ *
+ * @throws Exception
+ */
+ public void start() throws Exception;
+
+ /**
+ * Stop feed ingestion.
+ *
+ * @throws Exception
+ */
+ public void stop() throws Exception;
+
+ /**
+ * @return feedId associated with the feed that is being ingested
+ */
+ public FeedId getFeedId();
+
+ /**
+ * @return the instance of the feed adapter (an implementation of {@code IFeedAdapter}) in use.
+ */
+ public IFeedAdapter getFeedAdapter();
+
+ /**
+ * @return state associated with the AdapterRuntimeManager. See {@code State}.
+ */
+ public State getState();
+
+ /**
+ * @param state
+ */
+ public void setState(State state);
+
+ public IIntakeProgressTracker getProgressTracker();
+
+ public int getPartition();
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ICentralFeedManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ICentralFeedManager.java
new file mode 100644
index 0000000..b42335f
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ICentralFeedManager.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.asterix.common.feeds.api;
+
+import java.io.IOException;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+
+public interface ICentralFeedManager {
+
+ public void start() throws AsterixException;
+
+ public void stop() throws AsterixException, IOException;
+
+ public IFeedTrackingManager getFeedTrackingManager();
+
+ public IFeedLoadManager getFeedLoadManager();
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IDatasourceAdapter.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IDatasourceAdapter.java
similarity index 86%
rename from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IDatasourceAdapter.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IDatasourceAdapter.java
index a4c5de9..36ce7cf 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IDatasourceAdapter.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IDatasourceAdapter.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.metadata.feeds;
+package edu.uci.ics.asterix.common.feeds.api;
import java.io.Serializable;
@@ -27,12 +27,6 @@
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.
- */
-
- /**
* Triggers the adapter to begin ingesting data from the external source.
*
* @param partition
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IExceptionHandler.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IExceptionHandler.java
new file mode 100644
index 0000000..4e7e4e3
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IExceptionHandler.java
@@ -0,0 +1,39 @@
+/*
+ * 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.common.feeds.api;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Handles an exception encountered during processing of a data frame.
+ * In the case when the exception is of type {@code FrameDataException}, the causing
+ * tuple is logged and a new frame with tuple after the exception-generating tuple
+ * is returned. This funcitonality is used during feed ingestion to bypass an exception
+ * generating tuple and thus avoid the data flow from terminating
+ */
+public interface IExceptionHandler {
+
+ /**
+ * @param e
+ * the exception that needs to be handled
+ * @param frame
+ * the frame that was being processed when exception occurred
+ * @return returns a new frame with tuples after the exception generating tuple
+ * @throws HyracksDataException
+ */
+ public ByteBuffer handleException(Exception e, ByteBuffer frame);
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedAdapter.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedAdapter.java
new file mode 100644
index 0000000..bd2f6d3
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedAdapter.java
@@ -0,0 +1,56 @@
+/*
+ * 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.common.feeds.api;
+
+/**
+ * Interface implemented by a feed adapter.
+ */
+public interface IFeedAdapter extends IDatasourceAdapter {
+
+ public enum DataExchangeMode {
+ /**
+ * PULL model requires the adaptor to make a separate request each time to receive data
+ **/
+ PULL,
+
+ /**
+ * PUSH mode involves the use o just one initial request (handshake) by the adaptor
+ * to the datasource for setting up the connection and providing any protocol-specific
+ * parameters. Once a connection is established, the data source "pushes" data to the adaptor.
+ **/
+ PUSH
+ }
+
+ /**
+ * Returns the data exchange mode (PULL/PUSH) associated with the flow.
+ *
+ * @return
+ */
+ public DataExchangeMode getDataExchangeMode();
+
+ /**
+ * Discontinue the ingestion of data and end the feed.
+ *
+ * @throws Exception
+ */
+ public void stop() throws Exception;
+
+ /**
+ * @param e
+ * @return true if the feed ingestion should continue post the exception else false
+ */
+ public boolean handleException(Exception e);
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedConnectionManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedConnectionManager.java
new file mode 100644
index 0000000..0f91b52c
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedConnectionManager.java
@@ -0,0 +1,71 @@
+/*
+ * 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.common.feeds.api;
+
+import java.io.IOException;
+import java.util.List;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeManager;
+
+/**
+ * Handle (de)registration of feeds for delivery of control messages.
+ */
+public interface IFeedConnectionManager {
+
+ /**
+ * Allows registration of a feedRuntime.
+ *
+ * @param feedRuntime
+ * @throws Exception
+ */
+ public void registerFeedRuntime(FeedConnectionId connectionId, FeedRuntime feedRuntime) throws Exception;
+
+ /**
+ * Obtain feed runtime corresponding to a feedRuntimeId
+ *
+ * @param feedRuntimeId
+ * @return
+ */
+ public FeedRuntime getFeedRuntime(FeedConnectionId connectionId, FeedRuntimeId feedRuntimeId);
+
+ /**
+ * De-register a feed
+ *
+ * @param feedConnection
+ * @throws IOException
+ */
+ void deregisterFeed(FeedConnectionId feedConnection);
+
+ /**
+ * Obtain the feed runtime manager associated with a feed.
+ *
+ * @param feedConnection
+ * @return
+ */
+ public FeedRuntimeManager getFeedRuntimeManager(FeedConnectionId feedConnection);
+
+ /**
+ * Allows de-registration of a feed runtime.
+ *
+ * @param feedRuntimeId
+ */
+ void deRegisterFeedRuntime(FeedConnectionId connectionId, FeedRuntimeId feedRuntimeId);
+
+ public List<FeedRuntimeId> getRegisteredRuntimes();
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedFrameHandler.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedFrameHandler.java
new file mode 100644
index 0000000..057416a
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedFrameHandler.java
@@ -0,0 +1,35 @@
+/*
+ * 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.common.feeds.api;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+
+import edu.uci.ics.asterix.common.feeds.DataBucket;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFeedFrameHandler {
+
+ public void handleFrame(ByteBuffer frame) throws HyracksDataException;
+
+ public void handleDataBucket(DataBucket bucket);
+
+ public void close();
+
+ public Iterator<ByteBuffer> replayData() throws HyracksDataException;
+
+ public String getSummary();
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedJoint.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedJoint.java
new file mode 100644
index 0000000..0c52a42
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedJoint.java
@@ -0,0 +1,117 @@
+/*
+ * Copyright 2009-2014 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.common.feeds.api;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedJointKey;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionRequest;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleListener.ConnectionLocation;
+
+public interface IFeedJoint {
+
+ public enum FeedJointType {
+ /** Feed Joint is located at the intake stage of a primary feed **/
+ INTAKE,
+
+ /** Feed Joint is located at the compute stage of a primary/secondary feed **/
+ COMPUTE
+ }
+
+ public enum State {
+ /** Initial state of a feed joint post creation but prior to scheduling of corresponding Hyracks job. **/
+ CREATED,
+
+ /** State acquired post creation of Hyracks job and known physical locations of the joint **/
+ INITIALIZED,
+
+ /** State acquired post starting of Hyracks job at which point, data begins to flow through the joint **/
+ ACTIVE
+ }
+
+ /**
+ * @return the {@link State} associated with the FeedJoint
+ */
+ public State getState();
+
+ /**
+ * @return the {@link FeedJointType} associated with the FeedJoint
+ */
+ public FeedJointType getType();
+
+ /**
+ * @return the list of data receivers that are
+ * receiving the data flowing through this FeedJoint
+ */
+ public List<FeedConnectionId> getReceivers();
+
+ /**
+ * @return the list of pending subscription request {@link FeedConnectionRequest} submitted for data flowing through the FeedJoint
+ */
+ public List<FeedConnectionRequest> getConnectionRequests();
+
+ /**
+ * @return the subscription location {@link ConnectionLocation} associated with the FeedJoint
+ */
+ public ConnectionLocation getConnectionLocation();
+
+ /**
+ * @return the unique {@link FeedJointKey} associated with the FeedJoint
+ */
+ public FeedJointKey getFeedJointKey();
+
+ /**
+ * Returns the feed subscriber {@link FeedSubscriber} corresponding to a given feed connection id.
+ *
+ * @param feedConnectionId
+ * the unique id of a feed connection
+ * @return an instance of feedConnectionId {@link FeedConnectionId}
+ */
+ public FeedConnectionId getReceiver(FeedConnectionId feedConnectionId);
+
+ /**
+ * @param active
+ */
+ public void setState(State active);
+
+ /**
+ * Remove the subscriber from the set of registered subscribers to the FeedJoint
+ *
+ * @param connectionId
+ * the connectionId that needs to be removed
+ */
+ public void removeReceiver(FeedConnectionId connectionId);
+
+ public FeedId getOwnerFeedId();
+
+ /**
+ * Add a feed connectionId to the set of registered subscribers
+ *
+ * @param connectionId
+ */
+ public void addReceiver(FeedConnectionId connectionId);
+
+ /**
+ * Add a feed subscription request {@link FeedConnectionRequest} for the FeedJoint
+ *
+ * @param request
+ */
+ public void addConnectionRequest(FeedConnectionRequest request);
+
+ public FeedConnectionId getProvider();
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedLifecycleEventSubscriber.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedLifecycleEventSubscriber.java
new file mode 100644
index 0000000..84deec9
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedLifecycleEventSubscriber.java
@@ -0,0 +1,19 @@
+package edu.uci.ics.asterix.common.feeds.api;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedConnectJobInfo;
+
+public interface IFeedLifecycleEventSubscriber {
+
+ public enum FeedLifecycleEvent {
+ FEED_INTAKE_STARTED,
+ FEED_COLLECT_STARTED,
+ FEED_INTAKE_FAILURE,
+ FEED_COLLECT_FAILURE,
+ FEED_ENDED
+ }
+
+ public void assertEvent(FeedLifecycleEvent event) throws AsterixException, InterruptedException;
+
+ public void handleFeedEvent(FeedLifecycleEvent event);
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedLifecycleIntakeEventSubscriber.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedLifecycleIntakeEventSubscriber.java
new file mode 100644
index 0000000..f857156
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedLifecycleIntakeEventSubscriber.java
@@ -0,0 +1,10 @@
+package edu.uci.ics.asterix.common.feeds.api;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedIntakeInfo;
+
+public interface IFeedLifecycleIntakeEventSubscriber extends IFeedLifecycleEventSubscriber {
+
+ public void handleFeedEvent(FeedIntakeInfo iInfo, FeedLifecycleEvent event) throws AsterixException;
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedLifecycleListener.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedLifecycleListener.java
new file mode 100644
index 0000000..fb755c9
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedLifecycleListener.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.common.feeds.api;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.common.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedJointKey;
+import edu.uci.ics.hyracks.api.job.IJobLifecycleListener;
+
+public interface IFeedLifecycleListener extends IJobLifecycleListener, IClusterEventsSubscriber {
+
+ public enum ConnectionLocation {
+ SOURCE_FEED_INTAKE_STAGE,
+ SOURCE_FEED_COMPUTE_STAGE
+ }
+
+ public IFeedJoint getAvailableFeedJoint(FeedJointKey feedJoinKey);
+
+ public boolean isFeedJointAvailable(FeedJointKey feedJoinKey);
+
+ public List<FeedConnectionId> getActiveFeedConnections(FeedId feedId);
+
+ public List<String> getComputeLocations(FeedId feedId);
+
+ public List<String> getIntakeLocations(FeedId feedId);
+
+ public List<String> getStoreLocations(FeedConnectionId feedId);
+
+ public void registerFeedEventSubscriber(FeedConnectionId connectionId, IFeedLifecycleEventSubscriber subscriber);
+
+ public void deregisterFeedEventSubscriber(FeedConnectionId connectionId, IFeedLifecycleEventSubscriber subscriber);
+
+ public List<String> getCollectLocations(FeedConnectionId feedConnectionId);
+
+ boolean isFeedConnectionActive(FeedConnectionId connectionId);
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedLoadManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedLoadManager.java
new file mode 100644
index 0000000..bf9bbd0
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedLoadManager.java
@@ -0,0 +1,42 @@
+package edu.uci.ics.asterix.common.feeds.api;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.json.JSONException;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedActivity;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.NodeLoadReport;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.message.FeedCongestionMessage;
+import edu.uci.ics.asterix.common.feeds.message.FeedReportMessage;
+import edu.uci.ics.asterix.common.feeds.message.ScaleInReportMessage;
+import edu.uci.ics.asterix.common.feeds.message.ThrottlingEnabledFeedMessage;
+
+public interface IFeedLoadManager {
+
+ public void submitNodeLoadReport(NodeLoadReport report);
+
+ public void reportCongestion(FeedCongestionMessage message) throws JSONException, AsterixException;
+
+ public void submitFeedRuntimeReport(FeedReportMessage message);
+
+ public void submitScaleInPossibleReport(ScaleInReportMessage sm) throws AsterixException, Exception;
+
+ public List<String> getNodes(int required);
+
+ public void reportThrottlingEnabled(ThrottlingEnabledFeedMessage mesg) throws AsterixException, Exception;
+
+ int getOutflowRate(FeedConnectionId connectionId, FeedRuntimeType runtimeType);
+
+ void reportFeedActivity(FeedConnectionId connectionId, FeedActivity activity);
+
+ void removeFeedActivity(FeedConnectionId connectionId);
+
+ public FeedActivity getFeedActivity(FeedConnectionId connectionId);
+
+ public Collection<FeedActivity> getFeedActivities();
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedManager.java
new file mode 100644
index 0000000..a9368ae
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedManager.java
@@ -0,0 +1,93 @@
+/*
+ * 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.common.feeds.api;
+
+import edu.uci.ics.asterix.common.config.AsterixFeedProperties;
+import edu.uci.ics.asterix.common.feeds.api.IFeedConnectionManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessageService;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetadataManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector;
+import edu.uci.ics.asterix.common.feeds.api.IFeedSubscriptionManager;
+
+/**
+ * Provides access to services related to feed management within a node controller
+ */
+public interface IFeedManager {
+
+ /**
+ * gets the nodeId associated with the host node controller
+ *
+ * @return the nodeId associated with the host node controller
+ */
+ public String getNodeId();
+
+ /**
+ * gets the handle to the singleton instance of subscription manager
+ *
+ * @return the singleton instance of subscription manager
+ * @see IFeedSubscriptionManager
+ */
+ public IFeedSubscriptionManager getFeedSubscriptionManager();
+
+ /**
+ * gets the handle to the singleton instance of connection manager
+ *
+ * @return the singleton instance of connection manager
+ * @see IFeedConnectionManager
+ */
+ public IFeedConnectionManager getFeedConnectionManager();
+
+ /**
+ * gets the handle to the singleton instance of memory manager
+ *
+ * @return the singleton instance of memory manager
+ * @see IFeedMemoryManager
+ */
+ public IFeedMemoryManager getFeedMemoryManager();
+
+ /**
+ * gets the handle to the singleton instance of feed metadata manager
+ *
+ * @return the singleton instance of feed metadata manager
+ * @see IFeedMetadataManager
+ */
+ public IFeedMetadataManager getFeedMetadataManager();
+
+ /**
+ * gets the handle to the singleton instance of feed metric collector
+ *
+ * @return the singleton instance of feed metric collector
+ * @see IFeedMetricCollector
+ */
+ public IFeedMetricCollector getFeedMetricCollector();
+
+ /**
+ * gets the handle to the singleton instance of feed message service
+ *
+ * @return the singleton instance of feed message service
+ * @see IFeedMessageService
+ */
+ public IFeedMessageService getFeedMessageService();
+
+ /**
+ * gets the asterix configuration
+ *
+ * @return asterix configuration
+ * @see AsterixFeedProperties
+ */
+ public AsterixFeedProperties getAsterixFeedProperties();
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMemoryComponent.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMemoryComponent.java
new file mode 100644
index 0000000..da23ce9
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMemoryComponent.java
@@ -0,0 +1,54 @@
+/*
+ * 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.common.feeds.api;
+
+/**
+ * Represents an in-memory components required for storing frames that contain feed tuples.
+ * The component's memory footprint is measured and regulated by the {@link IFeedMemoryManager}.
+ * Any expansion in size is accounted and can be restricted by the {@link IFeedMemoryManager}
+ **/
+public interface IFeedMemoryComponent {
+
+ public enum Type {
+
+ /** A pool of reusable frames **/
+ POOL,
+
+ /** An ordered list of frames **/
+ COLLECTION
+ }
+
+ /** Gets the unique id associated with the memory component **/
+ public int getComponentId();
+
+ /** Gets the type associated with the component. **/
+ public Type getType();
+
+ /** Gets the current size (number of allocated frames) of the component. **/
+ public int getTotalAllocation();
+
+ /**
+ * Expands this memory component by the speficied number of frames
+ *
+ * @param delta
+ * the amount (measured in number of frames) by which this memory component
+ * should be expanded
+ */
+ public void expand(int delta);
+
+ /** Clears the allocated frames as a step to reclaim the memory **/
+ public void reset();
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMemoryManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMemoryManager.java
new file mode 100644
index 0000000..8b3322e
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMemoryManager.java
@@ -0,0 +1,54 @@
+/*
+ * 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.common.feeds.api;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedMemoryComponent.Type;
+
+/**
+ * Provides management of memory allocated for handling feed data flow through the node controller
+ */
+public interface IFeedMemoryManager {
+
+ public static final int START_COLLECTION_SIZE = 20;
+ public static final int START_POOL_SIZE = 10;
+
+ /**
+ * Gets a memory component allocated from the feed memory budget
+ *
+ * @param type
+ * the kind of memory component that needs to be allocated
+ * @return
+ * @see Type
+ */
+ public IFeedMemoryComponent getMemoryComponent(Type type);
+
+ /**
+ * Expand a memory component by the default increment
+ *
+ * @param memoryComponent
+ * @return true if the expansion succeeded
+ * false if the requested expansion violates the configured budget
+ */
+ public boolean expandMemoryComponent(IFeedMemoryComponent memoryComponent);
+
+ /**
+ * Releases the given memory component to reclaim the memory allocated for the component
+ *
+ * @param memoryComponent
+ * the memory component that is being reclaimed/released
+ */
+ public void releaseMemoryComponent(IFeedMemoryComponent memoryComponent);
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMessage.java
new file mode 100644
index 0000000..290b63e
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMessage.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.common.feeds.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.value.JSONSerializable;
+
+/**
+ * A control message exchanged between {@Link IFeedManager} and {@Link CentralFeedManager} that requests for an action or reporting of an event
+ */
+public interface IFeedMessage extends Serializable, JSONSerializable {
+
+ public enum MessageType {
+ END,
+ XAQL,
+ FEED_REPORT,
+ NODE_REPORT,
+ STORAGE_REPORT,
+ CONGESTION,
+ PREPARE_STALL,
+ TERMINATE_FLOW,
+ SCALE_IN_REQUEST,
+ COMMIT_ACK,
+ COMMIT_ACK_RESPONSE,
+ THROTTLING_ENABLED
+ }
+
+ /**
+ * Gets the type associated with this message
+ *
+ * @return MessageType type associated with this message
+ */
+ public MessageType getMessageType();
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMessageService.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMessageService.java
new file mode 100644
index 0000000..9086099
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMessageService.java
@@ -0,0 +1,30 @@
+/*
+ * 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.common.feeds.api;
+
+/**
+ * Provides the functionality of sending a meesage ({@code IFeedMessage} to the {@code CentralFeedManager}
+ */
+public interface IFeedMessageService extends IFeedService {
+
+ /**
+ * Sends a message ({@code IFeedMessage} to the {@code CentralFeedManager} running at the CC
+ * The message is sent asynchronously.
+ *
+ * @param message
+ * the message to be sent
+ */
+ public void sendMessage(IFeedMessage message);
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMetadataManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMetadataManager.java
new file mode 100644
index 0000000..fc0dd4e
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMetadataManager.java
@@ -0,0 +1,21 @@
+package edu.uci.ics.asterix.common.feeds.api;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+
+public interface IFeedMetadataManager {
+
+ /**
+ * @param feedConnectionId
+ * connection id corresponding to the feed connection
+ * @param tuple
+ * the erroneous tuple that raised an exception
+ * @param message
+ * the message corresponding to the exception being raised
+ * @param feedManager
+ * @throws AsterixException
+ */
+ public void logTuple(FeedConnectionId feedConnectionId, String tuple, String message, IFeedManager feedManager)
+ throws AsterixException;
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMetricCollector.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMetricCollector.java
new file mode 100644
index 0000000..1680387
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedMetricCollector.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.common.feeds.api;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+
+public interface IFeedMetricCollector {
+
+ public enum ValueType {
+ CPU_USAGE,
+ INFLOW_RATE,
+ OUTFLOW_RATE
+ }
+
+ public enum MetricType {
+ AVG,
+ RATE
+ }
+
+ public boolean sendReport(int senderId, int value);
+
+ public int getMetric(int senderId);
+
+ public int getMetric(FeedConnectionId connectionId, FeedRuntimeId runtimeId, ValueType valueType);
+
+ int createReportSender(FeedConnectionId connectionId, FeedRuntimeId runtimeId, ValueType valueType,
+ MetricType metricType);
+
+ public void removeReportSender(int senderId);
+
+ public void resetReportSender(int senderId);
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedOperatorOutputSideHandler.java
similarity index 60%
copy from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
copy to asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedOperatorOutputSideHandler.java
index 65ac354..d8c0b94 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedOperatorOutputSideHandler.java
@@ -12,18 +12,21 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.metadata.api;
+package edu.uci.ics.asterix.common.feeds.api;
-public interface IClusterManagementWork {
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
- public enum WorkType {
- ADD_NODE,
- REMOVE_NODE
+public interface IFeedOperatorOutputSideHandler extends IFrameWriter {
+
+ public enum Type {
+ BASIC_FEED_OUTPUT_HANDLER,
+ DISTRIBUTE_FEED_OUTPUT_HANDLER,
+ COLLECT_TRANSFORM_FEED_OUTPUT_HANDLER
}
- public WorkType getClusterManagementWorkType();
+ public FeedId getFeedId();
- public int getWorkId();
+ public Type getType();
- public IClusterEventsSubscriber getSourceSubscriber();
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedProvider.java
new file mode 100644
index 0000000..89dc33b
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedProvider.java
@@ -0,0 +1,8 @@
+package edu.uci.ics.asterix.common.feeds.api;
+
+import edu.uci.ics.asterix.common.feeds.FeedId;
+
+public interface IFeedProvider {
+
+ public void subscribeFeed(FeedId sourceDeedId);
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedRuntime.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedRuntime.java
new file mode 100644
index 0000000..73336b7
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedRuntime.java
@@ -0,0 +1,58 @@
+/*
+ * 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.common.feeds.api;
+
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeInputHandler;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+
+public interface IFeedRuntime {
+
+ public enum FeedRuntimeType {
+ INTAKE,
+ COLLECT,
+ COMPUTE_COLLECT,
+ COMPUTE,
+ STORE,
+ OTHER,
+ ETS,
+ JOIN
+ }
+
+ public enum Mode {
+ PROCESS,
+ SPILL,
+ PROCESS_SPILL,
+ DISCARD,
+ POST_SPILL_DISCARD,
+ PROCESS_BACKLOG,
+ STALL,
+ FAIL,
+ END
+ }
+
+ /**
+ * @return the unique runtime id associated with the feedRuntime
+ */
+ public FeedRuntimeId getRuntimeId();
+
+ /**
+ * @return the frame writer associated with the feed runtime.
+ */
+ public IFrameWriter getFeedFrameWriter();
+
+ public FeedRuntimeInputHandler getInputHandler();
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedService.java
similarity index 67%
copy from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
copy to asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedService.java
index 65ac354..6e85eca 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedService.java
@@ -12,18 +12,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.metadata.api;
+package edu.uci.ics.asterix.common.feeds.api;
-public interface IClusterManagementWork {
+public interface IFeedService {
- public enum WorkType {
- ADD_NODE,
- REMOVE_NODE
- }
+ public void start() throws Exception;
- public WorkType getClusterManagementWorkType();
-
- public int getWorkId();
-
- public IClusterEventsSubscriber getSourceSubscriber();
+ public void stop();
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedSubscriptionManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedSubscriptionManager.java
new file mode 100644
index 0000000..ea476f8
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedSubscriptionManager.java
@@ -0,0 +1,37 @@
+/*
+ * 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.common.feeds.api;
+
+import edu.uci.ics.asterix.common.feeds.SubscribableFeedRuntimeId;
+
+public interface IFeedSubscriptionManager {
+
+ /**
+ * @param subscribableRuntime
+ */
+ public void registerFeedSubscribableRuntime(ISubscribableRuntime subscribableRuntime);
+
+ /**
+ * @param subscribableRuntimeId
+ */
+ public void deregisterFeedSubscribableRuntime(SubscribableFeedRuntimeId subscribableRuntimeId);
+
+ /**
+ * @param subscribableRuntimeId
+ * @return
+ */
+ public ISubscribableRuntime getSubscribableRuntime(SubscribableFeedRuntimeId subscribableRuntimeId);
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedTrackingManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedTrackingManager.java
new file mode 100644
index 0000000..958bf23
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedTrackingManager.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.asterix.common.feeds.api;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedTupleCommitAckMessage;
+
+public interface IFeedTrackingManager {
+
+ public void submitAckReport(FeedTupleCommitAckMessage ackMessage);
+
+ public void disableAcking(FeedConnectionId connectionId);
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedWork.java
similarity index 67%
copy from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
copy to asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedWork.java
index 65ac354..8e340b5 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedWork.java
@@ -12,18 +12,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.metadata.api;
+package edu.uci.ics.asterix.common.feeds.api;
-public interface IClusterManagementWork {
+/**
+ * Represents a feed management task. The task is executed asynchronously.
+ */
+public interface IFeedWork {
- public enum WorkType {
- ADD_NODE,
- REMOVE_NODE
- }
+ public Runnable getRunnable();
- public WorkType getClusterManagementWorkType();
-
- public int getWorkId();
-
- public IClusterEventsSubscriber getSourceSubscriber();
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedWorkEventListener.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedWorkEventListener.java
new file mode 100644
index 0000000..4b989da
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedWorkEventListener.java
@@ -0,0 +1,37 @@
+/*
+ * 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.common.feeds.api;
+
+/**
+ * Provides a callback mechanism that in invoked for events related to
+ * the execution of a feed management task.
+ */
+public interface IFeedWorkEventListener {
+
+ /**
+ * A call back that is invoked after successful completion of a feed
+ * management task.
+ */
+ public void workCompleted(IFeedWork work);
+
+ /**
+ * A call back that is invokved after a failed execution of a feed
+ * management task.
+ *
+ * @param e
+ * exception encountered during execution of the task.
+ */
+ public void workFailed(IFeedWork work, Exception e);
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedWorkManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedWorkManager.java
new file mode 100644
index 0000000..58ea396
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFeedWorkManager.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.asterix.common.feeds.api;
+
+public interface IFeedWorkManager {
+
+ public void submitWork(IFeedWork work, IFeedWorkEventListener listener);
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFrameEventCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFrameEventCallback.java
new file mode 100644
index 0000000..0a1c1fe
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IFrameEventCallback.java
@@ -0,0 +1,14 @@
+package edu.uci.ics.asterix.common.feeds.api;
+
+public interface IFrameEventCallback {
+
+ public enum FrameEvent {
+ FINISHED_PROCESSING,
+ PENDING_WORK_THRESHOLD_REACHED,
+ PENDING_WORK_DONE,
+ NO_OP,
+ FINISHED_PROCESSING_SPILLAGE
+ }
+
+ public void frameEvent(FrameEvent frameEvent);
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IIntakeProgressTracker.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IIntakeProgressTracker.java
new file mode 100644
index 0000000..88f80e4
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IIntakeProgressTracker.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.asterix.common.feeds.api;
+
+import java.util.Map;
+
+public interface IIntakeProgressTracker {
+
+ public void configure(Map<String, String> configuration);
+
+ public void notifyIngestedTupleTimestamp(long timestamp);
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IMessageReceiver.java
similarity index 67%
copy from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
copy to asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IMessageReceiver.java
index 65ac354..4d986ab 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/IMessageReceiver.java
@@ -12,18 +12,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.metadata.api;
+package edu.uci.ics.asterix.common.feeds.api;
-public interface IClusterManagementWork {
+public interface IMessageReceiver<T> {
- public enum WorkType {
- ADD_NODE,
- REMOVE_NODE
- }
+ public void sendMessage(T message);
- public WorkType getClusterManagementWorkType();
+ public void close(boolean processPending);
- public int getWorkId();
-
- public IClusterEventsSubscriber getSourceSubscriber();
+ public void start();
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ISubscribableRuntime.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ISubscribableRuntime.java
new file mode 100644
index 0000000..3b44cbb
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ISubscribableRuntime.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.common.feeds.api;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.common.feeds.CollectionRuntime;
+import edu.uci.ics.asterix.common.feeds.DistributeFeedFrameWriter;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+/**
+ * Represent a feed runtime whose output can be routed along other parallel path(s).
+ */
+public interface ISubscribableRuntime extends IFeedRuntime {
+
+ /**
+ * @param collectionRuntime
+ * @throws Exception
+ */
+ public void subscribeFeed(FeedPolicyAccessor fpa, CollectionRuntime collectionRuntime) throws Exception;
+
+ /**
+ * @param collectionRuntime
+ * @throws Exception
+ */
+ public void unsubscribeFeed(CollectionRuntime collectionRuntime) throws Exception;
+
+ /**
+ * @return
+ * @throws Exception
+ */
+ public List<ISubscriberRuntime> getSubscribers();
+
+ /**
+ * @return
+ */
+ public DistributeFeedFrameWriter getFeedFrameWriter();
+
+ /**
+ * @return
+ */
+ public RecordDescriptor getRecordDescriptor();
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ISubscriberRuntime.java
similarity index 68%
copy from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
copy to asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ISubscriberRuntime.java
index 65ac354..abf913c 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ISubscriberRuntime.java
@@ -12,18 +12,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.metadata.api;
+package edu.uci.ics.asterix.common.feeds.api;
-public interface IClusterManagementWork {
+import java.util.Map;
- public enum WorkType {
- ADD_NODE,
- REMOVE_NODE
- }
+import edu.uci.ics.asterix.common.feeds.FeedFrameCollector;
- public WorkType getClusterManagementWorkType();
+public interface ISubscriberRuntime {
- public int getWorkId();
+ public Map<String, String> getFeedPolicy();
- public IClusterEventsSubscriber getSourceSubscriber();
+ public FeedFrameCollector getFrameCollector();
}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ISubscriptionProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ISubscriptionProvider.java
new file mode 100644
index 0000000..76593ea
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ISubscriptionProvider.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.asterix.common.feeds.api;
+
+import edu.uci.ics.asterix.common.feeds.FeedId;
+
+public interface ISubscriptionProvider {
+
+ public void subscribeFeed(FeedId sourceFeedId, FeedId recipientFeedId);
+
+ public void unsubscribeFeed(FeedId sourceFeedId, FeedId recipientFeedId);
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ITupleTrackingFeedAdapter.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ITupleTrackingFeedAdapter.java
new file mode 100644
index 0000000..485b9f0
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/api/ITupleTrackingFeedAdapter.java
@@ -0,0 +1,6 @@
+package edu.uci.ics.asterix.common.feeds.api;
+
+public interface ITupleTrackingFeedAdapter extends IFeedAdapter {
+
+ public void tuplePersistedTimeCallback(long timestamp);
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/EndFeedMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/EndFeedMessage.java
new file mode 100644
index 0000000..4d08ea3
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/EndFeedMessage.java
@@ -0,0 +1,93 @@
+/*
+ * 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.common.feeds.message;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+
+/**
+ * A feed control message indicating the need to end the feed. This message is dispatched
+ * to all locations that host an operator involved in the feed pipeline.
+ */
+public class EndFeedMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedId sourceFeedId;
+
+ private final FeedConnectionId connectionId;
+
+ private final FeedRuntimeType sourceRuntimeType;
+
+ private final boolean completeDisconnection;
+
+ private final EndMessageType endMessageType;
+
+ public enum EndMessageType {
+ DISCONNECT_FEED,
+ DISCONTINUE_SOURCE
+ }
+
+ public EndFeedMessage(FeedConnectionId connectionId, FeedRuntimeType sourceRuntimeType, FeedId sourceFeedId,
+ boolean completeDisconnection, EndMessageType endMessageType) {
+ super(MessageType.END);
+ this.connectionId = connectionId;
+ this.sourceRuntimeType = sourceRuntimeType;
+ this.sourceFeedId = sourceFeedId;
+ this.completeDisconnection = completeDisconnection;
+ this.endMessageType = endMessageType;
+ }
+
+ @Override
+ public String toString() {
+ return MessageType.END.name() + " " + connectionId + " [" + sourceRuntimeType + "] ";
+ }
+
+ public FeedRuntimeType getSourceRuntimeType() {
+ return sourceRuntimeType;
+ }
+
+ public FeedId getSourceFeedId() {
+ return sourceFeedId;
+ }
+
+ public boolean isCompleteDisconnection() {
+ return completeDisconnection;
+ }
+
+ public EndMessageType getEndMessageType() {
+ return endMessageType;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ return obj;
+ }
+
+ public FeedConnectionId getFeedConnectionId() {
+ return connectionId;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedCongestionMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedCongestionMessage.java
new file mode 100644
index 0000000..16fb115
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedCongestionMessage.java
@@ -0,0 +1,99 @@
+/*
+ * 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.common.feeds.message;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.FeedConstants.MessageConstants;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.Mode;
+
+public class FeedCongestionMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedConnectionId connectionId;
+ private final FeedRuntimeId runtimeId;
+ private int inflowRate;
+ private int outflowRate;
+ private Mode mode;
+
+ public FeedCongestionMessage(FeedConnectionId connectionId, FeedRuntimeId runtimeId, int inflowRate,
+ int outflowRate, Mode mode) {
+ super(MessageType.CONGESTION);
+ this.connectionId = connectionId;
+ this.runtimeId = runtimeId;
+ this.inflowRate = inflowRate;
+ this.outflowRate = outflowRate;
+ this.mode = mode;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ obj.put(FeedConstants.MessageConstants.RUNTIME_TYPE, runtimeId.getFeedRuntimeType());
+ obj.put(FeedConstants.MessageConstants.OPERAND_ID, runtimeId.getOperandId());
+ obj.put(FeedConstants.MessageConstants.PARTITION, runtimeId.getPartition());
+ obj.put(FeedConstants.MessageConstants.INFLOW_RATE, inflowRate);
+ obj.put(FeedConstants.MessageConstants.OUTFLOW_RATE, outflowRate);
+ obj.put(FeedConstants.MessageConstants.MODE, mode);
+ return obj;
+ }
+
+ public FeedRuntimeId getRuntimeId() {
+ return runtimeId;
+ }
+
+ public int getInflowRate() {
+ return inflowRate;
+ }
+
+ public int getOutflowRate() {
+ return outflowRate;
+ }
+
+ public static FeedCongestionMessage read(JSONObject obj) throws JSONException {
+ FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
+ obj.getString(FeedConstants.MessageConstants.FEED));
+ FeedConnectionId connectionId = new FeedConnectionId(feedId,
+ obj.getString(FeedConstants.MessageConstants.DATASET));
+ FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.valueOf(obj
+ .getString(FeedConstants.MessageConstants.RUNTIME_TYPE)),
+ obj.getInt(FeedConstants.MessageConstants.PARTITION),
+ obj.getString(FeedConstants.MessageConstants.OPERAND_ID));
+ Mode mode = Mode.valueOf(obj.getString(MessageConstants.MODE));
+ return new FeedCongestionMessage(connectionId, runtimeId,
+ obj.getInt(FeedConstants.MessageConstants.INFLOW_RATE),
+ obj.getInt(FeedConstants.MessageConstants.OUTFLOW_RATE), mode);
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public Mode getMode() {
+ return mode;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedMessage.java
new file mode 100644
index 0000000..62e774c
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedMessage.java
@@ -0,0 +1,38 @@
+/*
+ * 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.common.feeds.message;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessage;
+import edu.uci.ics.hyracks.api.dataflow.value.JSONSerializable;
+
+/**
+ * A control message that can be sent to the runtime instance of a
+ * feed's adapter.
+ */
+public abstract class FeedMessage implements IFeedMessage, JSONSerializable {
+
+ private static final long serialVersionUID = 1L;
+
+ protected final MessageType messageType;
+
+ public FeedMessage(MessageType messageType) {
+ this.messageType = messageType;
+ }
+
+ public MessageType getMessageType() {
+ return messageType;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedMessageService.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedMessageService.java
new file mode 100644
index 0000000..51a950b
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedMessageService.java
@@ -0,0 +1,143 @@
+/*
+ * 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.common.feeds.message;
+
+import java.net.Socket;
+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 org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.config.AsterixFeedProperties;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessage;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessageService;
+
+/**
+ * Sends feed report messages on behalf of an operator instance
+ * to the SuperFeedManager associated with the feed.
+ */
+public class FeedMessageService implements IFeedMessageService {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedMessageService.class.getName());
+
+ private final LinkedBlockingQueue<String> inbox;
+ private final FeedMessageHandler mesgHandler;
+ private final String nodeId;
+ private ExecutorService executor;
+
+ public FeedMessageService(AsterixFeedProperties feedProperties, String nodeId, String ccClusterIp) {
+ this.inbox = new LinkedBlockingQueue<String>();
+ this.mesgHandler = new FeedMessageHandler(inbox, ccClusterIp, feedProperties.getFeedCentralManagerPort());
+ this.nodeId = nodeId;
+ this.executor = Executors.newSingleThreadExecutor();
+ }
+
+ public void start() throws Exception {
+ executor.execute(mesgHandler);
+ }
+
+ public void stop() {
+ synchronized (mesgHandler.getLock()) {
+ executor.shutdownNow();
+ }
+ mesgHandler.stop();
+ }
+
+ @Override
+ public void sendMessage(IFeedMessage message) {
+ try {
+ JSONObject obj = message.toJSON();
+ obj.put(FeedConstants.MessageConstants.NODE_ID, nodeId);
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, message.getMessageType().name());
+ inbox.add(obj.toString());
+ } catch (JSONException jse) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("JSON Exception in parsing message " + message);
+ }
+ }
+ }
+
+ private static class FeedMessageHandler implements Runnable {
+
+ private final LinkedBlockingQueue<String> inbox;
+ private final String host;
+ private final int port;
+
+ private Socket cfmSocket;
+ private Object lock;
+
+ private static final byte[] EOL = "\n".getBytes();
+
+ public FeedMessageHandler(LinkedBlockingQueue<String> inbox, String host, int port) {
+ this.inbox = inbox;
+ this.host = host;
+ this.port = port;
+ this.lock = new Object();
+ }
+
+ public void run() {
+ try {
+ cfmSocket = new Socket(host, port);
+ if (cfmSocket != null) {
+ while (true) {
+ String message = inbox.take();
+ synchronized (lock) {
+ cfmSocket.getOutputStream().write(message.getBytes());
+ cfmSocket.getOutputStream().write(EOL);
+ }
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to start feed message service");
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Exception in handling incoming feed messages" + e.getMessage());
+ }
+ } finally {
+ stop();
+ }
+
+ }
+
+ public void stop() {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Stopping feed message handler");
+ }
+ if (cfmSocket != null) {
+ try {
+ cfmSocket.close();
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Exception in closing socket " + e.getMessage());
+ }
+ }
+ }
+ }
+
+ public Object getLock() {
+ return lock;
+ }
+
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedReportMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedReportMessage.java
new file mode 100644
index 0000000..128803a
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedReportMessage.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2009-2014 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.common.feeds.message;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.FeedConstants.MessageConstants;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMetricCollector.ValueType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+
+public class FeedReportMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedConnectionId connectionId;
+ private final FeedRuntimeId runtimeId;
+ private final ValueType valueType;
+ private int value;
+
+ public FeedReportMessage(FeedConnectionId connectionId, FeedRuntimeId runtimeId, ValueType valueType, int value) {
+ super(MessageType.FEED_REPORT);
+ this.connectionId = connectionId;
+ this.runtimeId = runtimeId;
+ this.valueType = valueType;
+ this.value = value;
+ }
+
+ public void reset(int value) {
+ this.value = value;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ obj.put(FeedConstants.MessageConstants.RUNTIME_TYPE, runtimeId.getFeedRuntimeType());
+ obj.put(FeedConstants.MessageConstants.PARTITION, runtimeId.getPartition());
+ obj.put(FeedConstants.MessageConstants.VALUE_TYPE, valueType);
+ obj.put(FeedConstants.MessageConstants.VALUE, value);
+ return obj;
+ }
+
+ public static FeedReportMessage read(JSONObject obj) throws JSONException {
+ FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
+ obj.getString(FeedConstants.MessageConstants.FEED));
+ FeedConnectionId connectionId = new FeedConnectionId(feedId,
+ obj.getString(FeedConstants.MessageConstants.DATASET));
+ FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.valueOf(obj
+ .getString(FeedConstants.MessageConstants.RUNTIME_TYPE)),
+ obj.getInt(FeedConstants.MessageConstants.PARTITION), FeedConstants.MessageConstants.NOT_APPLICABLE);
+ ValueType type = ValueType.valueOf(obj.getString(MessageConstants.VALUE_TYPE));
+ int value = Integer.parseInt(obj.getString(MessageConstants.VALUE));
+ return new FeedReportMessage(connectionId, runtimeId, type, value);
+ }
+
+ public int getValue() {
+ return value;
+ }
+
+ public void setValue(int value) {
+ this.value = value;
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public FeedRuntimeId getRuntimeId() {
+ return runtimeId;
+ }
+
+ public ValueType getValueType() {
+ return valueType;
+ }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedTupleCommitAckMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedTupleCommitAckMessage.java
new file mode 100644
index 0000000..cacfeb2
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedTupleCommitAckMessage.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.common.feeds.message;
+
+import javax.xml.bind.DatatypeConverter;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+
+public class FeedTupleCommitAckMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedConnectionId connectionId;
+ private int intakePartition;
+ private int base;
+ private byte[] commitAcks;
+
+ public FeedTupleCommitAckMessage(FeedConnectionId connectionId, int intakePartition, int base, byte[] commitAcks) {
+ super(MessageType.COMMIT_ACK);
+ this.connectionId = connectionId;
+ this.intakePartition = intakePartition;
+ this.base = base;
+ this.commitAcks = commitAcks;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ obj.put(FeedConstants.MessageConstants.INTAKE_PARTITION, intakePartition);
+ obj.put(FeedConstants.MessageConstants.BASE, base);
+ String commitAcksString = DatatypeConverter.printBase64Binary(commitAcks);
+ obj.put(FeedConstants.MessageConstants.COMMIT_ACKS, commitAcksString);
+ return obj;
+ }
+
+ public static FeedTupleCommitAckMessage read(JSONObject obj) throws JSONException {
+ FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
+ obj.getString(FeedConstants.MessageConstants.FEED));
+ FeedConnectionId connectionId = new FeedConnectionId(feedId,
+ obj.getString(FeedConstants.MessageConstants.DATASET));
+ int intakePartition = obj.getInt(FeedConstants.MessageConstants.INTAKE_PARTITION);
+ int base = obj.getInt(FeedConstants.MessageConstants.BASE);
+ String commitAcksString = obj.getString(FeedConstants.MessageConstants.COMMIT_ACKS);
+ byte[] commitAcks = DatatypeConverter.parseBase64Binary(commitAcksString);
+ return new FeedTupleCommitAckMessage(connectionId, intakePartition, base, commitAcks);
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public int getIntakePartition() {
+ return intakePartition;
+ }
+
+ public byte[] getCommitAcks() {
+ return commitAcks;
+ }
+
+ public void reset(int intakePartition, int base, byte[] commitAcks) {
+ this.intakePartition = intakePartition;
+ this.base = base;
+ this.commitAcks = commitAcks;
+ }
+
+ public int getBase() {
+ return base;
+ }
+
+ public void setBase(int base) {
+ this.base = base;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedTupleCommitResponseMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedTupleCommitResponseMessage.java
new file mode 100644
index 0000000..663a629
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/FeedTupleCommitResponseMessage.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.common.feeds.message;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+
+public class FeedTupleCommitResponseMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedConnectionId connectionId;
+ private final int intakePartition;
+ private final int maxWindowAcked;
+
+ public FeedTupleCommitResponseMessage(FeedConnectionId connectionId, int intakePartition, int maxWindowAcked) {
+ super(MessageType.COMMIT_ACK_RESPONSE);
+ this.connectionId = connectionId;
+ this.intakePartition = intakePartition;
+ this.maxWindowAcked = maxWindowAcked;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ obj.put(FeedConstants.MessageConstants.INTAKE_PARTITION, intakePartition);
+ obj.put(FeedConstants.MessageConstants.MAX_WINDOW_ACKED, maxWindowAcked);
+ return obj;
+ }
+
+ public static FeedTupleCommitResponseMessage read(JSONObject obj) throws JSONException {
+ FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
+ obj.getString(FeedConstants.MessageConstants.FEED));
+ FeedConnectionId connectionId = new FeedConnectionId(feedId,
+ obj.getString(FeedConstants.MessageConstants.DATASET));
+ int intakePartition = obj.getInt(FeedConstants.MessageConstants.INTAKE_PARTITION);
+ int maxWindowAcked = obj.getInt(FeedConstants.MessageConstants.MAX_WINDOW_ACKED);
+ return new FeedTupleCommitResponseMessage(connectionId, intakePartition, maxWindowAcked);
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public int getMaxWindowAcked() {
+ return maxWindowAcked;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/NodeReportMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/NodeReportMessage.java
new file mode 100644
index 0000000..32c35f3
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/NodeReportMessage.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 2009-2014 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.common.feeds.message;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessage;
+
+public class NodeReportMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private double cpuLoad;
+ private double usedHeap;
+ private int nRuntimes;
+
+ public NodeReportMessage(float cpuLoad, long usedHeap, int nRuntimes) {
+ super(IFeedMessage.MessageType.NODE_REPORT);
+ this.usedHeap = usedHeap;
+ this.cpuLoad = cpuLoad;
+ this.nRuntimes = nRuntimes;
+ }
+
+ public void reset(double cpuLoad, double usedHeap, int nRuntimes) {
+ this.cpuLoad = cpuLoad;
+ this.usedHeap = usedHeap;
+ this.nRuntimes = nRuntimes;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.CPU_LOAD, cpuLoad);
+ obj.put(FeedConstants.MessageConstants.HEAP_USAGE, usedHeap);
+ obj.put(FeedConstants.MessageConstants.N_RUNTIMES, nRuntimes);
+ return obj;
+ }
+
+ public double getCpuLoad() {
+ return cpuLoad;
+ }
+
+ public double getUsedHeap() {
+ return usedHeap;
+ }
+
+ public int getnRuntimes() {
+ return nRuntimes;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/ScaleInReportMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/ScaleInReportMessage.java
new file mode 100644
index 0000000..7acc20f
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/ScaleInReportMessage.java
@@ -0,0 +1,110 @@
+/*
+ * 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.common.feeds.message;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+
+/**
+ * A feed control message indicating the need to scale in a stage of the feed ingestion pipeline.
+ * Currently, scaling-in of the compute stage is supported.
+ **/
+public class ScaleInReportMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedConnectionId connectionId;
+
+ private final FeedRuntimeType runtimeType;
+
+ private int currentCardinality;
+
+ private int reducedCardinaliy;
+
+ public ScaleInReportMessage(FeedConnectionId connectionId, FeedRuntimeType runtimeType, int currentCardinality,
+ int reducedCardinaliy) {
+ super(MessageType.SCALE_IN_REQUEST);
+ this.connectionId = connectionId;
+ this.runtimeType = runtimeType;
+ this.currentCardinality = currentCardinality;
+ this.reducedCardinaliy = reducedCardinaliy;
+ }
+
+ @Override
+ public String toString() {
+ return MessageType.SCALE_IN_REQUEST.name() + " " + connectionId + " [" + runtimeType + "] "
+ + " currentCardinality " + currentCardinality + " reducedCardinality " + reducedCardinaliy;
+ }
+
+ public FeedRuntimeType getRuntimeType() {
+ return runtimeType;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ obj.put(FeedConstants.MessageConstants.RUNTIME_TYPE, runtimeType);
+ obj.put(FeedConstants.MessageConstants.CURRENT_CARDINALITY, currentCardinality);
+ obj.put(FeedConstants.MessageConstants.REDUCED_CARDINALITY, reducedCardinaliy);
+ return obj;
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public static ScaleInReportMessage read(JSONObject obj) throws JSONException {
+ FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
+ obj.getString(FeedConstants.MessageConstants.FEED));
+ FeedConnectionId connectionId = new FeedConnectionId(feedId,
+ obj.getString(FeedConstants.MessageConstants.DATASET));
+ FeedRuntimeType runtimeType = FeedRuntimeType.valueOf(obj
+ .getString(FeedConstants.MessageConstants.RUNTIME_TYPE));
+ return new ScaleInReportMessage(connectionId, runtimeType,
+ obj.getInt(FeedConstants.MessageConstants.CURRENT_CARDINALITY),
+ obj.getInt(FeedConstants.MessageConstants.REDUCED_CARDINALITY));
+ }
+
+ public void reset(int currentCardinality, int reducedCardinaliy) {
+ this.currentCardinality = currentCardinality;
+ this.reducedCardinaliy = reducedCardinaliy;
+ }
+
+ public int getCurrentCardinality() {
+ return currentCardinality;
+ }
+
+ public void setCurrentCardinality(int currentCardinality) {
+ this.currentCardinality = currentCardinality;
+ }
+
+ public int getReducedCardinaliy() {
+ return reducedCardinaliy;
+ }
+
+ public void setReducedCardinaliy(int reducedCardinaliy) {
+ this.reducedCardinaliy = reducedCardinaliy;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/StorageReportFeedMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/StorageReportFeedMessage.java
new file mode 100644
index 0000000..502eb98
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/StorageReportFeedMessage.java
@@ -0,0 +1,125 @@
+/*
+ * 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.common.feeds.message;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.FeedConstants.MessageConstants;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+
+/**
+ * A feed control message sent from a storage runtime of a feed pipeline to report the intake timestamp corresponding
+ * to the last persisted tuple.
+ */
+public class StorageReportFeedMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedConnectionId connectionId;
+ private final int partition;
+ private long lastPersistedTupleIntakeTimestamp;
+ private boolean persistenceDelayWithinLimit;
+ private long averageDelay;
+ private int intakePartition;
+
+ public StorageReportFeedMessage(FeedConnectionId connectionId, int partition,
+ long lastPersistedTupleIntakeTimestamp, boolean persistenceDelayWithinLimit, long averageDelay,
+ int intakePartition) {
+ super(MessageType.STORAGE_REPORT);
+ this.connectionId = connectionId;
+ this.partition = partition;
+ this.lastPersistedTupleIntakeTimestamp = lastPersistedTupleIntakeTimestamp;
+ this.persistenceDelayWithinLimit = persistenceDelayWithinLimit;
+ this.averageDelay = averageDelay;
+ this.intakePartition = intakePartition;
+ }
+
+ @Override
+ public String toString() {
+ return messageType.name() + " " + connectionId + " [" + lastPersistedTupleIntakeTimestamp + "] ";
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public long getLastPersistedTupleIntakeTimestamp() {
+ return lastPersistedTupleIntakeTimestamp;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+
+ public boolean isPersistenceDelayWithinLimit() {
+ return persistenceDelayWithinLimit;
+ }
+
+ public void setPersistenceDelayWithinLimit(boolean persistenceDelayWithinLimit) {
+ this.persistenceDelayWithinLimit = persistenceDelayWithinLimit;
+ }
+
+ public long getAverageDelay() {
+ return averageDelay;
+ }
+
+ public void setAverageDelay(long averageDelay) {
+ this.averageDelay = averageDelay;
+ }
+
+ public int getIntakePartition() {
+ return intakePartition;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ obj.put(FeedConstants.MessageConstants.LAST_PERSISTED_TUPLE_INTAKE_TIMESTAMP, lastPersistedTupleIntakeTimestamp);
+ obj.put(MessageConstants.PERSISTENCE_DELAY_WITHIN_LIMIT, persistenceDelayWithinLimit);
+ obj.put(MessageConstants.AVERAGE_PERSISTENCE_DELAY, averageDelay);
+ obj.put(FeedConstants.MessageConstants.PARTITION, partition);
+ obj.put(FeedConstants.MessageConstants.INTAKE_PARTITION, intakePartition);
+
+ return obj;
+ }
+
+ public static StorageReportFeedMessage read(JSONObject obj) throws JSONException {
+ FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
+ obj.getString(FeedConstants.MessageConstants.FEED));
+ FeedConnectionId connectionId = new FeedConnectionId(feedId,
+ obj.getString(FeedConstants.MessageConstants.DATASET));
+ int partition = obj.getInt(FeedConstants.MessageConstants.PARTITION);
+ long timestamp = obj.getLong(FeedConstants.MessageConstants.LAST_PERSISTED_TUPLE_INTAKE_TIMESTAMP);
+ boolean persistenceDelayWithinLimit = obj.getBoolean(MessageConstants.PERSISTENCE_DELAY_WITHIN_LIMIT);
+ long averageDelay = obj.getLong(MessageConstants.AVERAGE_PERSISTENCE_DELAY);
+ int intakePartition = obj.getInt(MessageConstants.INTAKE_PARTITION);
+ return new StorageReportFeedMessage(connectionId, partition, timestamp, persistenceDelayWithinLimit,
+ averageDelay, intakePartition);
+ }
+
+ public void reset(long lastPersistedTupleIntakeTimestamp, boolean delayWithinLimit, long averageDelay) {
+ this.lastPersistedTupleIntakeTimestamp = lastPersistedTupleIntakeTimestamp;
+ this.persistenceDelayWithinLimit = delayWithinLimit;
+ this.averageDelay = averageDelay;
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/ThrottlingEnabledFeedMessage.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/ThrottlingEnabledFeedMessage.java
new file mode 100644
index 0000000..4bb750b
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/feeds/message/ThrottlingEnabledFeedMessage.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.common.feeds.message;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+
+/**
+ * A feed control message indicating the need to end the feed. This message is dispatched
+ * to all locations that host an operator involved in the feed pipeline.
+ */
+public class ThrottlingEnabledFeedMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedConnectionId connectionId;
+
+ private final FeedRuntimeId runtimeId;
+
+ public ThrottlingEnabledFeedMessage(FeedConnectionId connectionId, FeedRuntimeId runtimeId) {
+ super(MessageType.THROTTLING_ENABLED);
+ this.connectionId = connectionId;
+ this.runtimeId = runtimeId;
+ }
+
+ @Override
+ public String toString() {
+ return MessageType.END.name() + " " + connectionId + " [" + runtimeId + "] ";
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ obj.put(FeedConstants.MessageConstants.RUNTIME_TYPE, runtimeId.getFeedRuntimeType());
+ obj.put(FeedConstants.MessageConstants.OPERAND_ID, runtimeId.getOperandId());
+ obj.put(FeedConstants.MessageConstants.PARTITION, runtimeId.getPartition());
+ return obj;
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public FeedRuntimeId getFeedRuntimeId() {
+ return runtimeId;
+ }
+
+ public static ThrottlingEnabledFeedMessage read(JSONObject obj) throws JSONException {
+ FeedId feedId = new FeedId(obj.getString(FeedConstants.MessageConstants.DATAVERSE),
+ obj.getString(FeedConstants.MessageConstants.FEED));
+ FeedConnectionId connectionId = new FeedConnectionId(feedId,
+ obj.getString(FeedConstants.MessageConstants.DATASET));
+ FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.valueOf(obj
+ .getString(FeedConstants.MessageConstants.RUNTIME_TYPE)),
+ obj.getInt(FeedConstants.MessageConstants.PARTITION),
+ obj.getString(FeedConstants.MessageConstants.OPERAND_ID));
+ return new ThrottlingEnabledFeedMessage(connectionId, runtimeId);
+ }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/parse/IAsterixTupleParser.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/parse/IAsterixTupleParser.java
new file mode 100644
index 0000000..0a03d79
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/parse/IAsterixTupleParser.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.asterix.common.parse;
+
+import java.util.Map;
+
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+
+public interface IAsterixTupleParser extends ITupleParser{
+
+ public void configure(Map<String, String> configuration);
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/parse/ITupleForwardPolicy.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/parse/ITupleForwardPolicy.java
new file mode 100644
index 0000000..e7fdc74
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/parse/ITupleForwardPolicy.java
@@ -0,0 +1,30 @@
+package edu.uci.ics.asterix.common.parse;
+
+import java.util.Map;
+
+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;
+
+public interface ITupleForwardPolicy {
+
+ public static final String PARSER_POLICY = "parser-policy";
+
+ public enum TupleForwardPolicyType {
+ FRAME_FULL,
+ COUNTER_TIMER_EXPIRED,
+ RATE_CONTROLLED
+ }
+
+ public void configure(Map<String, String> configuration);
+
+ public void initialize(IHyracksTaskContext ctx, IFrameWriter frameWriter) throws HyracksDataException;
+
+ public TupleForwardPolicyType getType();
+
+ public void addTuple(ArrayTupleBuilder tb) throws HyracksDataException;
+
+ public void close() throws HyracksDataException;
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/parse/ITupleParserPolicy.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/parse/ITupleParserPolicy.java
new file mode 100644
index 0000000..652139c
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/parse/ITupleParserPolicy.java
@@ -0,0 +1,27 @@
+package edu.uci.ics.asterix.common.parse;
+
+import java.util.Map;
+
+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;
+
+public interface ITupleParserPolicy {
+
+ public enum TupleParserPolicy {
+ FRAME_FULL,
+ TIME_COUNT_ELAPSED,
+ RATE_CONTROLLED
+ }
+
+ public TupleParserPolicy getType();
+
+ public void configure(Map<String, String> configuration) throws HyracksDataException;
+
+ public void initialize(IHyracksTaskContext ctx, IFrameWriter writer) throws HyracksDataException;
+
+ public void addTuple(ArrayTupleBuilder tb) throws HyracksDataException;
+
+ public void close() throws HyracksDataException;
+}
diff --git a/asterix-doc/src/site/markdown/feeds/tutorial.md b/asterix-doc/src/site/markdown/feeds/tutorial.md
new file mode 100644
index 0000000..5884449
--- /dev/null
+++ b/asterix-doc/src/site/markdown/feeds/tutorial.md
@@ -0,0 +1,287 @@
+# Support for Data Ingestion in AsterixDB #
+
+## <a id="toc">Table of Contents</a> ##
+
+* [Introduction](#Introduction)
+ * [Data Feed Basics](#DataFeedBasics)
+ * [Collecting Data: Feed Adaptors](#FeedAdaptors)
+ * [Preprocessing Collected Data](#PreprocessingCollectedData)
+ * [Creating an External Dataset](#IntroductionCreatingAnExternalDataset)
+
+## <a id="Introduction">Introduction</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
+In this document, we describe the support for data ingestion in AsterixDB, an open-source Big Data Management System (BDMS) that provides a platform for storage and analysis of large volumes of semi-structured data. Data feeds are a new mechanism for having
+continuous data arrive into a BDMS from external sources and incrementally populate a persisted dataset and associated indexes. We add a new BDMS architectural component, called a data feed, that makes a Big Data system the caretaker for functionality that
+used to live outside, and we show how it improves users’ lives and system performance.
+
+### <a id="DataFeedBasics">Data Feed Basics</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ###
+ ####Collecting Data: Feed Adaptors####
+The functionality of establishing a connection with a data source
+and receiving, parsing and translating its data into ADM records
+(for storage inside AsterixDB) is contained in a feed adaptor. A
+feed adaptor is an implementation of an interface and its details are
+specific to a given data source. An adaptor may optionally be given
+parameters to configure its runtime behavior. Depending upon the
+data transfer protocol/APIs offered by the data source, a feed adaptor
+may operate in a push or a pull mode. Push mode involves just
+one initial request by the adaptor to the data source for setting up
+the connection. Once a connection is authorized, the data source
+“pushes” data to the adaptor without any subsequent requests by
+the adaptor. In contrast, when operating in a pull mode, the adaptor
+makes a separate request each time to receive data.
+AsterixDB currently provides built-in adaptors for several popular
+data sources—Twitter, CNN, and RSS feeds. AsterixDB additionally
+provides a generic socket-based adaptor that can be used
+to ingest data that is directed at a prescribed socket.
+
+Next, we consider creating an example feed that consists of tweets obtained from
+the Twitter service. To do so, we use the built-in push-based Twitter adaptor.
+To being with, we must define a Tweet using the AsterixDB Data Model (ADM) and the AsterixDB Query Language (AQL). Given below are the type definitions in AQL that create a Tweet datatype which is representative of a real tweet as obtained from Twitter.
+
+ create dataverse feeds;
+ use dataverse feeds;
+
+ create type TwitterUser if not exists as open{
+ screen_name: string,
+ language: string,
+ friends_count: int32,
+ status_count: int32,
+ name: string,
+ followers_count: string
+ };
+ create type Tweet if not exists as open{
+ id: string,
+ user: TwitterUser,
+ latitude:double,
+ longitude:double,
+ created_at:string,
+ message_text:string
+ };
+
+
+Next we make use of the create feed AQL statement to define our example data feed.
+
+ create feed TwitterFeed if not exists using "push_twitter"
+ (("type-name"="Tweet"),("location"="US"));
+
+Note that the create feed statement does not initiate the flow of data from Twitter into our AsterixDB instance. Instead, the create feed statement only results in registering the feed with AsterixDB. The flow of data along a feed is initiated when it is connected
+to a target dataset using the connect feed statement (which we shall revisit later).
+
+ ####Preprocessing Collected Data####
+A feed definition may optionally include the specification of a
+user-defined function that is to be applied to each feed record prior
+to persistence. Examples of pre-processing might include adding
+attributes, filtering out records, sampling, sentiment analysis, feature
+extraction, etc. The pre-processing is expressed as a userdefined
+function (UDF) that can be defined in AQL or in a programming
+language like Java. An AQL UDF is a good fit when
+pre-processing a record requires the result of a query (join or aggregate)
+over data contained in AsterixDB datasets. More sophisticated
+processing such as sentiment analysis of text is better handled
+by providing a Java UDF. A Java UDF has an initialization phase
+that allows the UDF to access any resources it may need to initialize
+itself prior to being used in a data flow. It is assumed by the
+AsterixDB compiler to be stateless and thus usable as an embarassingly
+parallel black box. In constrast, the AsterixDB compiler can
+reason about an AQL UDF and involve the use of indexes during
+its invocation.
+
+The tweets collected by the Twitter Adaptor (push_twiiter) (Figure 4) conform
+to the Tweet datatype (defined earlier). We consider an example transformation of a raw tweet into its lightweight version - ProcessedTweet - which is defined next.
+
+ create type ProcessedTweet if not exists as open {
+ id: string,
+ user_name:string,
+ location:point,
+ created_at:string,
+ message_text:string,
+ country: string,
+ topics: [string]
+ };
+
+
+The processing required in transforming a collected tweet to its lighter version (of type ProcessedTweet) involves extracting the topicsor hash-tags (if any) in a tweet
+and collecting them in the referred-topics attribute for the tweet.
+Additionally, the latitude and longitude values (doubles) are combined into the spatial point type. Note that spatial data types are considered as first class citizens that come with the support for creating indexes. Next we show a revised version of our example TwitterFeed that involves the use of a UDF. We assume that the UDF that contains the transformation logic into a ProcessedTweet is avaialable as a Java UDF inside an AsterixDB library named 'testlib'. We defer the writing of a Java UDF and its installation as part of an AsterixDB library to a later section of this document.
+
+ create feed ProcessedTwitterFeed if not exists
+ using "push_twitter"
+ (("type-name"="Tweet"),("location"="US"));
+ apply function testlib#processRawTweet;
+
+Note that a feed adaptor and a UDF act as pluggable components. These
+contribute towards providing a generic ‘plug-and-play‘ model where
+custom implementations can be provided to cater to specific requirements.
+
+####Building a Cascade Network of Feeds####
+Multiple high-level applications may wish to consume the data
+ingested from a data feed. Each such application might perceive the
+feed in a different way and require the arriving data to be processed
+and/or persisted differently. Building a separate flow of data from
+the external source for each application is wasteful of resources as
+the pre-processing or transformations required by each application
+might overlap and could be done together in an incremental fashion
+to avoid redundancy. A single flow of data from the external source
+could provide data for multiple applications. To achieve this, we
+introduce the notion of primary and secondary feeds in AsterixDB.
+
+A feed in AsterixDB is considered to be a primary feed if it gets
+its data from an external data source. The records contained in a
+feed (subsequent to any pre-processing) are directed to a designated
+AsterixDB dataset. Alternatively or additionally, these records can
+be used to derive other feeds known as secondary feeds. A secondary
+feed is similar to its parent feed in every other aspect; it can
+have an associated UDF to allow for any subsequent processing,
+can be persisted into a dataset, and/or can be made to derive other
+secondary feeds to form a cascade network. A primary feed and a
+dependent secondary feed form a hierarchy. As an example, we next show an
+example AQL statement that redefines the previous feed—
+ProcessedTwitterFeed in terms of their
+respective parent feed (TwitterFeed).
+
+ create secondary feed ProcessedTwitterFeed from feed TwitterFeed
+ apply function testlib#addFeatures;
+
+
+####Lifecycle of a Feed####
+A feed is a logical artifact that is brought to life (i.e. its data flow
+is initiated) only when it is connected to a dataset using the connect
+feed AQL statement (Figure 7). Subsequent to a connect feed
+statement, the feed is said to be in the connected state. Multiple
+feeds can simultaneously be connected to a dataset such that the
+contents of the dataset represent the union of the connected feeds.
+In a supported but unlikely scenario, one feed may also be simultaneously
+connected to different target datasets. Note that connecting
+a secondary feed does not require the parent feed (or any ancestor
+feed) to be in the connected state; the order in which feeds are connected
+to their respective datasets is not important. Furthermore,
+additional (secondary) feeds can be added to an existing hierarchy
+and connected to a dataset at any time without impeding/interrupting
+the flow of data along a connected ancestor feed.
+
+ connect feed ProcessedTwitterFeed to
+ dataset ProcessedTweets ;
+
+ disconnect feed ProcessedTwitterFeed from
+ dataset ProcessedTweets ;
+
+The connect feed statement above directs AsterixDB to persist
+the ProcessedTwitterFeed feed in the ProcessedTweets dataset.
+If it is required (by the high-level application) to also retain the raw
+tweets obtained from Twitter, the end user may additionally choose
+to connect TwitterFeed to a (different) dataset. Having a set of primary
+and secondary feeds offers the flexibility to do so. Let us
+assume that the application needs to persist TwitterFeed and that,
+to do so, the end user makes another use of the connect feed statement.
+A logical view of the continuous flow of data established by
+connecting the feeds to their respective target datasets is shown in
+Figure 8.
+
+The flow of data from a feed into a dataset can be terminated
+explicitly by use of the disconnect feed statement.
+Disconnecting a feed from a particular dataset does not interrupt
+the flow of data from the feed to any other dataset(s), nor does it
+impact other connected feeds in the lineage.
+
+####Policies for Feed Ingestion####
+Multiple feeds may be concurrently operational on an AsterixDB
+cluster, each competing for resources (CPU cycles, network bandwidth,
+disk IO) to maintain pace with their respective data sources.
+A data management system must be able to manage a set of concurrent
+feeds and make dynamic decisions related to the allocation of
+resources, resolving resource bottlenecks and the handling of failures.
+Each feed has its own set of constraints, influenced largely
+by the nature of its data source and the application(s) that intend
+to consume and process the ingested data. Consider an application
+that intends to discover the trending topics on Twitter by analyzing
+the ProcessedTwitterFeed feed. Losing a few tweets may be
+acceptable. In contrast, when ingesting from a data source that
+provides a click-stream of ad clicks, losing data would translate to
+a loss of revenue for an application that tracks revenue by charging
+advertisers per click.
+
+AsterixDB allows a data feed to have an associated ingestion
+policy that is expressed as a collection of parameters and associated
+values. An ingestion policy dictates the runtime behavior of
+the feed in response to resource bottlenecks and failures. AsterixDB provides
+a list of policy parameters (Table 1) that help customize the
+system’s runtime behavior when handling excess records. AsterixDB
+provides a set of built-in policies, each constructed by setting
+appropriate value(s) for the policy parameter(s) from the table below.
+
+Policy Parameter | Description | Default Value
+------------------|------------|---------------|
+excess.records.spill | Set to true if records that cannot be processed by an operator for lack of resources (referred to as excess records hereafter) should be persisted to the local disk for deferred processing. | false |
+excess.records.discard | Set to true if excess records should be discarded. | false |
+excess.records.throttle | Set to true if rate of arrival of records is required to be reduced in an adaptive manner to prevent having any excess records. | false |
+excess.records.elastic | Set to true if the system should attempt to resolve resource bottlenecks by re-structuring and/or rescheduling the feed ingestion pipeline. | false |
+recover.soft.failure | Set to true if the feed must attempt to survive any runtime exception. A false value permits an early termination of a feed in such an event. | true |
+recover.hard.failure | Set to true if the feed must attempt to survive a hardware failures (loss of AsterixDB node(s)). A false value permits the early termination of a feed in the event of a hardware failure | |
+
+Note that the end user may choose to form a custom policy. E.g.
+it is possible in AsterixDB to create a custom policy that spills excess
+records to disk and subsequently resorts to throttling if the
+spillage crosses a configured threshold. In all cases, the desired
+ingestion policy is specified as part of the connect feed statement
+(Figure 9) or else the ‘Basic’ policy will be chosen as the default.
+It is worth noting that a feed can be connected to a dataset at any
+time, which is independent from other related feeds in the hierarchy.
+
+
+ connect feed TwitterFeed to dataset Tweets
+ using policy Basic ;
+
+
+####Writing an External UDF####
+A Java UDF in AsterixDB is required to implement an prescribe interface. We shall next write a basic UDF that extracts the hashtags contained in the tweet's text and appends each into an unordered list. The list is added as an additional attribute to the tweet to form the augment version - ProcessedTweet.
+
+ package edu.uci.ics.asterix.external.library;
+
+ 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.JUnorderedList;
+ import edu.uci.ics.asterix.external.library.java.JTypeTag;
+
+ public class HashTagsFunction implements IExternalScalarFunction {
+
+ private JUnorderedList list = null;
+
+ @Override
+ public void initialize(IFunctionHelper functionHelper) {
+ list = new JUnorderedList(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 text = (JString) inputRecord.getValueByName("message_text");
+
+ // extraction of hashtags
+ 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);
+ }
+ }
+
+ // forming the return value - an augmented tweet with an additional attribute - topics
+ JRecord result = (JRecord) functionHelper.getResultObject();
+ result.setField("tweetid", inputRecord.getFields()[0]);
+ result.setField("user", inputRecord.getFields()[1]);
+ result.setField("location_lat", inputRecord.getFields()[2]);
+ result.setField("location_long", inputRecord.getFields()[3]);
+ result.setField("send_time", inputRecord.getFields()[4]);
+ result.setField("message_text", inputRecord.getFields()[5]);
+ result.setField("topics", list);
+
+ functionHelper.setResult(result);
+ }
+
+
+
diff --git a/asterix-external-data/pom.xml b/asterix-external-data/pom.xml
index 0b045f1..7d64b61 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -136,9 +136,14 @@
<dependency>
<groupId>org.twitter4j</groupId>
<artifactId>twitter4j-core</artifactId>
- <version>2.2.3</version>
+ <version>[3.0,)</version>
</dependency>
<dependency>
+ <groupId>org.twitter4j</groupId>
+ <artifactId>twitter4j-stream</artifactId>
+ <version>4.0.2</version>
+ </dependency>
+ <dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<type>jar</type>
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 fa66715..e4f118f 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
@@ -19,12 +19,12 @@
import java.util.List;
import java.util.Map;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
import edu.uci.ics.asterix.external.dataset.adapter.RSSFeedAdapter;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
-import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapterFactory;
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;
@@ -32,7 +32,7 @@
/**
* A factory class for creating the @see {CNNFeedAdapter}.
*/
-public class CNNFeedAdapterFactory implements ITypedAdapterFactory {
+public class CNNFeedAdapterFactory implements IFeedAdapterFactory {
private static final long serialVersionUID = 1L;
private Map<String, String> configuration;
@@ -40,6 +40,7 @@
private List<String> feedURLs = new ArrayList<String>();
private static Map<String, String> topicFeeds = new HashMap<String, String>();
private ARecordType recordType;
+ private FeedPolicyAccessor policyAccessor;
public static final String KEY_RSS_URL = "topic";
public static final String KEY_INTERVAL = "interval";
@@ -91,22 +92,14 @@
}
@Override
- public AdapterType getAdapterType() {
- return AdapterType.TYPED;
- }
-
- @Override
- public void configure(Map<String, String> configuration) throws Exception {
+ public void configure(Map<String, String> configuration, ARecordType outputType) 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);
-
+ this.recordType = outputType;
}
private void initializeFeedURLs(String rssURLProperty) {
@@ -147,4 +140,14 @@
return recordType;
}
+ @Override
+ public boolean isRecordTrackingEnabled() {
+ return false;
+ }
+
+ @Override
+ public IIntakeProgressTracker createIntakeProgressTracker() {
+ return null;
+ }
+
}
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 dbf3fd7..ee481de 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
@@ -31,15 +31,17 @@
import org.apache.hadoop.mapred.JobConf;
import edu.uci.ics.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
import edu.uci.ics.asterix.external.indexing.dataflow.HDFSObjectTupleParserFactory;
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
-import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory;
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.asterix.runtime.operators.file.AsterixTupleParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
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;
@@ -53,7 +55,7 @@
/**
* A factory class for creating an instance of HDFSAdapter
*/
-public class HDFSAdapterFactory extends StreamBasedAdapterFactory implements IGenericAdapterFactory {
+public class HDFSAdapterFactory extends StreamBasedAdapterFactory implements IAdapterFactory {
private static final long serialVersionUID = 1L;
public static final String HDFS_ADAPTER_NAME = "hdfs";
@@ -80,6 +82,7 @@
private boolean configured = false;
public static Scheduler hdfsScheduler;
private static boolean initialized = false;
+ protected List<ExternalFile> files;
private static Scheduler initializeHDFSScheduler() {
ICCContext ccContext = AsterixAppContextInfo.getInstance().getCCApplicationContext().getCCContext();
@@ -137,11 +140,6 @@
}
@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");
@@ -203,25 +201,41 @@
return new AlgebricksAbsolutePartitionConstraint(cluster);
}
+ @Override
+ public ARecordType getAdapterOutputType() {
+ return (ARecordType) atype;
+ }
+
+ @Override
+ public InputDataFormat getInputDataFormat() {
+ return InputDataFormat.UNKNOWN;
+ }
+
/*
* This method is overridden to do the following:
* if data is text data (adm or delimited text), it will use a text tuple parser,
* otherwise it will use hdfs record object parser
*/
protected void configureFormat(IAType sourceDatatype) throws Exception {
- 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, false);
- } else if (FORMAT_ADM.equalsIgnoreCase((String) configuration.get(KEY_FORMAT))) {
- parserFactory = getADMDataTupleParserFactory((ARecordType) sourceDatatype, false);
- } else if (FORMAT_BINARY.equalsIgnoreCase((String) configuration.get(KEY_FORMAT))) {
- parserFactory = new HDFSObjectTupleParserFactory((ARecordType) atype, this, configuration);
- } else {
- throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
- }
- }
+ String specifiedFormat = (String) configuration.get(AsterixTupleParserFactory.KEY_FORMAT);
+ if (specifiedFormat == null) {
+ throw new IllegalArgumentException(" Unspecified data format");
+ }
+
+ if(AsterixTupleParserFactory.FORMAT_BINARY.equalsIgnoreCase(specifiedFormat)){
+ parserFactory = new HDFSObjectTupleParserFactory((ARecordType) atype, this, configuration);
+ } else {
+ InputDataFormat inputFormat = InputDataFormat.UNKNOWN;
+ if (AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
+ inputFormat = InputDataFormat.DELIMITED;
+ } else if (AsterixTupleParserFactory.FORMAT_ADM.equalsIgnoreCase(specifiedFormat)) {
+ inputFormat = InputDataFormat.ADM;
+ }
+ parserFactory = new AsterixTupleParserFactory(configuration, (ARecordType) sourceDatatype
+ , inputFormat);
+ }
+
+ }
/**
* Instead of creating the split using the input format, we do it manually
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSIndexingAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSIndexingAdapterFactory.java
index 2a4836c..37b8050 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSIndexingAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSIndexingAdapterFactory.java
@@ -16,22 +16,24 @@
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.HDFSIndexingAdapter;
import edu.uci.ics.asterix.external.indexing.dataflow.HDFSIndexingParserFactory;
import edu.uci.ics.asterix.external.indexing.dataflow.IndexingScheduler;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
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.IAType;
import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory;
import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -39,7 +41,12 @@
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.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.hdfs.dataflow.ConfFactory;
import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
@@ -83,11 +90,6 @@
}
@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");
@@ -104,7 +106,8 @@
((HDFSIndexingParserFactory) parserFactory).setArguments(configuration);
HDFSIndexingAdapter hdfsIndexingAdapter = new HDFSIndexingAdapter(atype, readSchedule, executed, inputSplits,
conf, clusterLocations, files, parserFactory, ctx, nodeName,
- (String) configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT), (String) configuration.get(KEY_FORMAT));
+ (String) configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT),
+ (String) configuration.get(AsterixTupleParserFactory.KEY_FORMAT));
return hdfsIndexingAdapter;
}
@@ -131,12 +134,13 @@
protected void configureFormat(IAType sourceDatatype) throws Exception {
- char delimiter = StreamBasedAdapterFactory.getDelimiter(configuration);
- char quote = StreamBasedAdapterFactory.getQuote(configuration, delimiter);
+ char delimiter = AsterixTupleParserFactory.getDelimiter(configuration);
+ char quote = AsterixTupleParserFactory.getQuote(configuration, delimiter);
parserFactory = new HDFSIndexingParserFactory((ARecordType) atype,
- configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT), configuration.get(KEY_FORMAT), delimiter,
- quote, configuration.get(HDFSAdapterFactory.KEY_PARSER));
+ configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT),
+ configuration.get(AsterixTupleParserFactory.KEY_FORMAT), delimiter, quote,
+ configuration.get(HDFSAdapterFactory.KEY_PARSER));
}
/**
@@ -165,7 +169,7 @@
if (tag == null) {
throw new NotImplementedException("Failed to get the type information for field " + i + ".");
}
- IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
+ IValueParserFactory vpf = valueParserFactoryMap.get(tag);
if (vpf == null) {
throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
}
@@ -190,4 +194,16 @@
cluster = locs.toArray(cluster);
return new AlgebricksAbsolutePartitionConstraint(cluster);
}
+
+ private static Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = initializeValueParserFactoryMap();
+
+ private static Map<ATypeTag, IValueParserFactory> initializeValueParserFactoryMap() {
+ Map<ATypeTag, IValueParserFactory> m = new HashMap<ATypeTag, IValueParserFactory>();
+ m.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
+ m.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
+ m.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
+ m.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
+ m.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
+ return m;
+ }
}
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 ab59241..c5ebe67 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
@@ -17,20 +17,22 @@
import java.util.List;
import java.util.Map;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter;
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
-import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
/**
* A factory class for creating an instance of HiveAdapter
*/
-public class HiveAdapterFactory extends StreamBasedAdapterFactory implements IGenericAdapterFactory {
+public class HiveAdapterFactory extends StreamBasedAdapterFactory implements IAdapterFactory {
private static final long serialVersionUID = 1L;
public static final String HIVE_DATABASE = "database";
@@ -61,11 +63,7 @@
return "hive";
}
- @Override
- public AdapterType getAdapterType() {
- return AdapterType.GENERIC;
- }
-
+
@Override
public SupportedOperation getSupportedOperations() {
return SupportedOperation.READ;
@@ -76,6 +74,7 @@
if (!configured) {
populateConfiguration(configuration);
hdfsAdapterFactory.configure(configuration, outputType);
+ this.atype = (IAType) outputType;
}
}
@@ -90,8 +89,8 @@
+ configuration.get(HIVE_TABLE);
}
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");
+ if (!configuration.get(AsterixTupleParserFactory.KEY_FORMAT).equals(AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT)) {
+ throw new IllegalArgumentException("format" + configuration.get(AsterixTupleParserFactory.KEY_FORMAT) + " is not supported");
}
if (!(configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT) || configuration
@@ -106,9 +105,19 @@
return hdfsAdapterFactory.getPartitionConstraint();
}
+
@Override
+ public ARecordType getAdapterOutputType() {
+ return (ARecordType) atype;
+ }
+
+ @Override
+ public InputDataFormat getInputDataFormat() {
+ return InputDataFormat.UNKNOWN;
+ }
+
public void setFiles(List<ExternalFile> files) {
- this.files = files;
+ hdfsAdapterFactory.setFiles(files);
}
}
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 110a965..446199e 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
@@ -20,15 +20,17 @@
import java.util.logging.Level;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
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.entities.ExternalFile;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
-import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -41,7 +43,7 @@
* NCFileSystemAdapter reads external data residing on the local file system of
* an NC.
*/
-public class NCFileSystemAdapterFactory extends StreamBasedAdapterFactory implements IGenericAdapterFactory {
+public class NCFileSystemAdapterFactory extends StreamBasedAdapterFactory implements IAdapterFactory {
private static final long serialVersionUID = 1L;
public static final String NC_FILE_SYSTEM_ADAPTER_NAME = "localfs";
@@ -50,6 +52,8 @@
private IAType sourceDatatype;
private FileSplit[] fileSplits;
+ private ARecordType outputType;
+
@Override
public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
@@ -62,10 +66,6 @@
return NC_FILE_SYSTEM_ADAPTER_NAME;
}
- @Override
- public AdapterType getAdapterType() {
- return AdapterType.GENERIC;
- }
@Override
public SupportedOperation getSupportedOperations() {
@@ -75,7 +75,8 @@
@Override
public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
this.configuration = configuration;
- String[] splits = ((String) configuration.get(KEY_PATH)).split(",");
+ this.outputType = outputType;
+ String[] splits = ((String) configuration.get(AsterixTupleParserFactory.KEY_PATH)).split(",");
IAType sourceDatatype = (IAType) outputType;
configureFileSplits(splits);
configureFormat(sourceDatatype);
@@ -127,7 +128,7 @@
private static INodeResolver initializeNodeResolver() {
INodeResolver nodeResolver = null;
- String configuredNodeResolverFactory = System.getProperty(NODE_RESOLVER_FACTORY_PROPERTY);
+ String configuredNodeResolverFactory = System.getProperty(AsterixTupleParserFactory.NODE_RESOLVER_FACTORY_PROPERTY);
if (configuredNodeResolverFactory != null) {
try {
nodeResolver = ((INodeResolverFactory) (Class.forName(configuredNodeResolverFactory).newInstance()))
@@ -145,8 +146,17 @@
}
return nodeResolver;
}
-
+
@Override
+ public ARecordType getAdapterOutputType() {
+ return outputType;
+ }
+
+ @Override
+ public InputDataFormat getInputDataFormat() {
+ return InputDataFormat.UNKNOWN;
+ }
+
public void setFiles(List<ExternalFile> files) throws AlgebricksException {
throw new AlgebricksException("can't set files for this Adapter");
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedAzureTwitterAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedAzureTwitterAdapterFactory.java
index b4dbe13..298a443 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedAzureTwitterAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PullBasedAzureTwitterAdapterFactory.java
@@ -3,12 +3,14 @@
import java.util.Map;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
import edu.uci.ics.asterix.external.dataset.adapter.PullBasedAzureTwitterAdapter;
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.feeds.IDatasourceAdapter;
-import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapterFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.IAType;
@@ -16,7 +18,7 @@
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-public class PullBasedAzureTwitterAdapterFactory implements ITypedAdapterFactory {
+public class PullBasedAzureTwitterAdapterFactory implements IFeedAdapterFactory {
private static final long serialVersionUID = 1L;
@@ -27,13 +29,14 @@
private static final String ACCOUNT_NAME_KEY = "account-name";
private static final String ACCOUNT_KEY_KEY = "account-key";
- private ARecordType recordType;
+ private ARecordType outputType;
private Map<String, String> configuration;
private String tableName;
private String azureAccountName;
private String azureAccountKey;
private String[] locations;
private String[] partitions;
+ private FeedPolicyAccessor ingestionPolicy;
@Override
public SupportedOperation getSupportedOperations() {
@@ -46,11 +49,6 @@
}
@Override
- public AdapterType getAdapterType() {
- return AdapterType.TYPED;
- }
-
- @Override
public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
String locationsStr = configuration.get(INGESTOR_LOCATIONS_KEY);
if (locationsStr == null) {
@@ -63,17 +61,18 @@
@Override
public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
return new PullBasedAzureTwitterAdapter(azureAccountName, azureAccountKey, tableName, partitions,
- configuration, ctx, recordType);
+ configuration, ctx, outputType);
}
@Override
public ARecordType getAdapterOutputType() {
- return recordType;
+ return outputType;
}
@Override
- public void configure(Map<String, String> configuration) throws Exception {
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
this.configuration = configuration;
+ this.outputType = outputType;
tableName = configuration.get(TABLE_NAME_KEY);
if (tableName == null) {
@@ -125,7 +124,7 @@
if (type.getTypeTag() != ATypeTag.RECORD) {
throw new IllegalStateException();
}
- recordType = (ARecordType) t.getDatatype();
+ outputType = (ARecordType) t.getDatatype();
MetadataManager.INSTANCE.commitTransaction(ctx);
} catch (Exception e) {
if (ctx != null) {
@@ -136,4 +135,21 @@
MetadataManager.INSTANCE.releaseReadLatch();
}
}
+
+ @Override
+ public boolean isRecordTrackingEnabled() {
+ return false;
+ }
+
+ @Override
+ public IIntakeProgressTracker createIntakeProgressTracker() {
+ return null;
+ }
+
+ public FeedPolicyAccessor getIngestionPolicy() {
+ return ingestionPolicy;
+ }
+
+
+
}
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 6058bd2..c43ca05 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
@@ -15,13 +15,18 @@
package edu.uci.ics.asterix.external.adapter.factory;
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.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
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.external.util.TwitterUtil;
+import edu.uci.ics.asterix.external.util.TwitterUtil.SearchAPIConstants;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapterFactory;
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;
@@ -31,29 +36,22 @@
* This adapter provides the functionality of fetching tweets from Twitter service
* via pull-based Twitter API.
*/
-public class PullBasedTwitterAdapterFactory implements ITypedAdapterFactory {
+public class PullBasedTwitterAdapterFactory implements IFeedAdapterFactory {
private static final long serialVersionUID = 1L;
+ private static final Logger LOGGER = Logger.getLogger(PullBasedTwitterAdapterFactory.class.getName());
+
public static final String PULL_BASED_TWITTER_ADAPTER_NAME = "pull_twitter";
- private Map<String, String> configuration;
- private static ARecordType recordType = initOutputType();
+ private static final String DEFAULT_INTERVAL = "10"; // 10 seconds
+ private static final int INTAKE_CARDINALITY = 1; // degree of parallelism at intake stage
- private static ARecordType initOutputType() {
- ARecordType 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("TweetType", fieldNames, fieldTypes, false);
- } catch (Exception e) {
- throw new IllegalStateException("Unable to create adapter output type");
- }
- return recordType;
- }
+ private ARecordType outputType;
+
+ private Map<String, String> configuration;
@Override
public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
- return new PullBasedTwitterAdapter(configuration, recordType, ctx);
+ return new PullBasedTwitterAdapter(configuration, outputType, ctx);
}
@Override
@@ -62,28 +60,57 @@
}
@Override
- public AdapterType getAdapterType() {
- return AdapterType.TYPED;
- }
-
- @Override
public SupportedOperation getSupportedOperations() {
return SupportedOperation.READ;
}
@Override
- public void configure(Map<String, String> configuration) throws Exception {
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ this.outputType = outputType;
this.configuration = configuration;
+ TwitterUtil.initializeConfigurationWithAuthInfo(configuration);
+
+ if (configuration.get(SearchAPIConstants.QUERY) == null) {
+ throw new AsterixException("parameter " + SearchAPIConstants.QUERY
+ + " not specified as part of adaptor configuration");
+ }
+
+ String interval = configuration.get(SearchAPIConstants.INTERVAL);
+ if (interval != null) {
+ try {
+ Integer.parseInt(interval);
+ } catch (NumberFormatException nfe) {
+ throw new IllegalArgumentException("parameter " + SearchAPIConstants.INTERVAL
+ + " is defined incorrectly, expecting a number");
+ }
+ } else {
+ configuration.put(SearchAPIConstants.INTERVAL, DEFAULT_INTERVAL);
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning(" Parameter " + SearchAPIConstants.INTERVAL + " not defined, using default ("
+ + DEFAULT_INTERVAL + ")");
+ }
+ }
+
}
@Override
public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
- return new AlgebricksCountPartitionConstraint(1);
+ return new AlgebricksCountPartitionConstraint(INTAKE_CARDINALITY);
+ }
+
+ @Override
+ public boolean isRecordTrackingEnabled() {
+ return false;
+ }
+
+ @Override
+ public IIntakeProgressTracker createIntakeProgressTracker() {
+ return null;
}
@Override
public ARecordType getAdapterOutputType() {
- return recordType;
+ return outputType;
}
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PushBasedTwitterAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PushBasedTwitterAdapterFactory.java
new file mode 100644
index 0000000..c947a6a
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/PushBasedTwitterAdapterFactory.java
@@ -0,0 +1,69 @@
+package edu.uci.ics.asterix.external.adapter.factory;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
+import edu.uci.ics.asterix.external.dataset.adapter.PushBasedTwitterAdapter;
+import edu.uci.ics.asterix.external.util.TwitterUtil;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapterFactory;
+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;
+
+public class PushBasedTwitterAdapterFactory implements IFeedAdapterFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final String NAME = "push_twitter";
+
+ private ARecordType outputType;
+
+ private Map<String, String> configuration;
+
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
+ }
+
+ @Override
+ public String getName() {
+ return NAME;
+ }
+
+ @Override
+ public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+ return new AlgebricksCountPartitionConstraint(1);
+ }
+
+ @Override
+ public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
+ PushBasedTwitterAdapter twitterAdapter = new PushBasedTwitterAdapter(configuration, outputType, ctx);
+ return twitterAdapter;
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ this.outputType = outputType;
+ this.configuration = configuration;
+ TwitterUtil.initializeConfigurationWithAuthInfo(configuration);
+ }
+
+ @Override
+ public ARecordType getAdapterOutputType() {
+ return outputType;
+ }
+
+
+ @Override
+ public boolean isRecordTrackingEnabled() {
+ return false;
+ }
+
+ @Override
+ public IIntakeProgressTracker createIntakeProgressTracker() {
+ return null;
+ }
+
+}
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 41f1d56..01ae2f2 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
@@ -18,12 +18,12 @@
import java.util.List;
import java.util.Map;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
import edu.uci.ics.asterix.external.dataset.adapter.RSSFeedAdapter;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
-import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapterFactory;
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;
@@ -32,7 +32,7 @@
* Factory class for creating an instance of @see {RSSFeedAdapter}.
* RSSFeedAdapter provides the functionality of fetching an RSS based feed.
*/
-public class RSSFeedAdapterFactory implements ITypedAdapterFactory {
+public class RSSFeedAdapterFactory implements IFeedAdapterFactory {
private static final long serialVersionUID = 1L;
public static final String RSS_FEED_ADAPTER_NAME = "rss_feed";
@@ -40,12 +40,13 @@
public static final String KEY_INTERVAL = "interval";
private Map<String, String> configuration;
- private ARecordType recordType;
+ private ARecordType outputType;
private List<String> feedURLs = new ArrayList<String>();
+ private FeedPolicyAccessor ingestionPolicy;
@Override
public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
- RSSFeedAdapter rssFeedAdapter = new RSSFeedAdapter(configuration, recordType, ctx);
+ RSSFeedAdapter rssFeedAdapter = new RSSFeedAdapter(configuration, outputType, ctx);
return rssFeedAdapter;
}
@@ -55,28 +56,20 @@
}
@Override
- public AdapterType getAdapterType() {
- return AdapterType.TYPED;
- }
-
- @Override
public SupportedOperation getSupportedOperations() {
return SupportedOperation.READ;
}
@Override
- public void configure(Map<String, String> configuration) throws Exception {
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
this.configuration = configuration;
+ this.outputType = outputType;
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
@@ -98,7 +91,21 @@
@Override
public ARecordType getAdapterOutputType() {
- return recordType;
+ return outputType;
+ }
+
+ @Override
+ public boolean isRecordTrackingEnabled() {
+ return false;
+ }
+
+ @Override
+ public IIntakeProgressTracker createIntakeProgressTracker() {
+ return null;
+ }
+
+ public FeedPolicyAccessor getIngestionPolicy() {
+ return ingestionPolicy;
}
}
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
index 0f340bc..7d45448 100644
--- 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
@@ -14,30 +14,15 @@
*/
package edu.uci.ics.asterix.external.adapter.factory;
-import java.util.HashMap;
-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.external.util.INodeResolver;
-import edu.uci.ics.asterix.metadata.entities.ExternalFile;
-import edu.uci.ics.asterix.metadata.feeds.ConditionalPushTupleParserFactory;
-import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory;
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.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.asterix.runtime.operators.file.AsterixTupleParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
public abstract class StreamBasedAdapterFactory implements IAdapterFactory {
@@ -45,146 +30,18 @@
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_QUOTE = "quote";
- public static final String KEY_HEADER = "header";
- public static final String KEY_PATH = "path";
- public static final String KEY_SOURCE_DATATYPE = "output-type-name";
- // The length of a delimiter should be 1.
- public static final String DEFAULT_DELIMITER = ",";
- // A quote is used to enclose a string if it includes delimiter(s) in it.
- // The length of a quote should be 1.
- public static final String DEFAULT_QUOTE = "\"";
- 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 Map<String, String> configuration;
protected ITupleParserFactory parserFactory;
- protected ITupleParser parser;
+
- protected List<ExternalFile> files;
-
- 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 = 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;
- }
-
- char delimiter = getDelimiter(configuration);
- char quote = getQuote(configuration, delimiter);
- boolean hasHeader = getHasHeader(configuration);
-
- return conditionalPush ? new ConditionalPushTupleParserFactory(recordType, fieldParserFactories, delimiter,
- quote, hasHeader, configuration) : new NtDelimitedDataTupleParserFactory(recordType,
- fieldParserFactories, delimiter, quote, hasHeader);
- }
-
- 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);
- }
-
- }
+ public abstract InputDataFormat getInputDataFormat();
protected void configureFormat(IAType sourceDatatype) throws Exception {
- String propValue = (String) configuration.get(BATCH_SIZE);
- int batchSize = propValue != null ? Integer.parseInt(propValue) : -1;
- propValue = configuration.get(BATCH_INTERVAL);
- long batchInterval = propValue != null ? Long.parseLong(propValue) : -1;
- boolean conditionalPush = batchSize > 0 || batchInterval > 0;
-
- String parserFactoryClassname = configuration.get(KEY_PARSER_FACTORY);
- if (parserFactoryClassname == null) {
- String specifiedFormat = 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();
- }
+ parserFactory = new AsterixTupleParserFactory(configuration, (ARecordType) sourceDatatype, getInputDataFormat());
}
- // Get a delimiter from the given configuration
- public static char getDelimiter(Map<String, String> configuration) throws AsterixException {
- String delimiterValue = configuration.get(KEY_DELIMITER);
- if (delimiterValue == null) {
- delimiterValue = DEFAULT_DELIMITER;
- } else if (delimiterValue.length() != 1) {
- throw new AsterixException("'" + delimiterValue
- + "' is not a valid delimiter. The length of a delimiter should be 1.");
- }
- return delimiterValue.charAt(0);
- }
-
- // Get a quote from the given configuration when the delimiter is given
- // Need to pass delimiter to check whether they share the same character
- public static char getQuote(Map<String, String> configuration, char delimiter) throws AsterixException {
- String quoteValue = configuration.get(KEY_QUOTE);
- if (quoteValue == null) {
- quoteValue = DEFAULT_QUOTE;
- } else if (quoteValue.length() != 1) {
- throw new AsterixException("'" + quoteValue + "' is not a valid quote. The length of a quote should be 1.");
- }
-
- // Since delimiter (char type value) can't be null,
- // we only check whether delimiter and quote use the same character
- if (quoteValue.charAt(0) == delimiter) {
- throw new AsterixException("Quote '" + quoteValue + "' cannot be used with the delimiter '" + delimiter
- + "'. ");
- }
-
- return quoteValue.charAt(0);
- }
-
- // Get the header flag
- public static boolean getHasHeader(Map<String, String> configuration) {
- return Boolean.parseBoolean(configuration.get(KEY_HEADER));
- }
+
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ClientBasedFeedAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ClientBasedFeedAdapter.java
new file mode 100644
index 0000000..4712e7c
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/ClientBasedFeedAdapter.java
@@ -0,0 +1,150 @@
+/*
+ * 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.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedAdapter;
+import edu.uci.ics.asterix.common.parse.ITupleForwardPolicy;
+import edu.uci.ics.asterix.external.dataset.adapter.IFeedClient.InflowState;
+import edu.uci.ics.asterix.metadata.feeds.FeedPolicyEnforcer;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+
+/**
+ * Acts as an abstract class for all pull-based external data adapters. Captures
+ * the common logic for obtaining bytes from an external source and packing them
+ * into frames as tuples.
+ */
+public abstract class ClientBasedFeedAdapter implements IFeedAdapter {
+
+ private static final long serialVersionUID = 1L;
+ private static final Logger LOGGER = Logger.getLogger(ClientBasedFeedAdapter.class.getName());
+ private static final int timeout = 5; // seconds
+
+ protected ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(1);
+ protected IFeedClient pullBasedFeedClient;
+ protected ARecordType adapterOutputType;
+ protected boolean continueIngestion = true;
+ protected Map<String, String> configuration;
+
+ private FrameTupleAppender appender;
+ private IFrame frame;
+ private long tupleCount = 0;
+ private final IHyracksTaskContext ctx;
+ private int frameTupleCount = 0;
+
+ protected FeedPolicyEnforcer policyEnforcer;
+
+ public FeedPolicyEnforcer getPolicyEnforcer() {
+ return policyEnforcer;
+ }
+
+ public void setFeedPolicyEnforcer(FeedPolicyEnforcer policyEnforcer) {
+ this.policyEnforcer = policyEnforcer;
+ }
+
+ public abstract IFeedClient getFeedClient(int partition) throws Exception;
+
+ public abstract ITupleForwardPolicy getTupleParserPolicy();
+
+ public ClientBasedFeedAdapter(Map<String, String> configuration, IHyracksTaskContext ctx) {
+ this.ctx = ctx;
+ this.configuration = configuration;
+ }
+
+ public long getIngestedRecordsCount() {
+ return tupleCount;
+ }
+
+ @Override
+ public void start(int partition, IFrameWriter writer) throws Exception {
+ appender = new FrameTupleAppender();
+ frame = new VSizeFrame(ctx);
+ appender.reset(frame, true);
+ ITupleForwardPolicy policy = getTupleParserPolicy();
+ policy.configure(configuration);
+ pullBasedFeedClient = getFeedClient(partition);
+ InflowState inflowState = null;
+ policy.initialize(ctx, writer);
+ while (continueIngestion) {
+ tupleBuilder.reset();
+ try {
+ inflowState = pullBasedFeedClient.nextTuple(tupleBuilder.getDataOutput(), timeout);
+ switch (inflowState) {
+ case DATA_AVAILABLE:
+ tupleBuilder.addFieldEndOffset();
+ policy.addTuple(tupleBuilder);
+ frameTupleCount++;
+ break;
+ case NO_MORE_DATA:
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Reached end of feed");
+ }
+ policy.close();
+ tupleCount += frameTupleCount;
+ frameTupleCount = 0;
+ continueIngestion = false;
+ break;
+ case DATA_NOT_AVAILABLE:
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Timed out on obtaining data from pull based adapter. Trying again!");
+ }
+ break;
+ }
+
+ } catch (Exception failureException) {
+ try {
+ failureException.printStackTrace();
+ boolean continueIngestion = policyEnforcer.continueIngestionPostSoftwareFailure(failureException);
+ if (continueIngestion) {
+ tupleBuilder.reset();
+ continue;
+ } else {
+ throw failureException;
+ }
+ } catch (Exception recoveryException) {
+ throw new Exception(recoveryException);
+ }
+ }
+ }
+ }
+
+ /**
+ * 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;
+ }
+
+ @Override
+ public boolean handleException(Exception e) {
+ return false;
+ }
+
+}
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/FeedClient.java
similarity index 82%
rename from asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedFeedClient.java
rename to asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/FeedClient.java
index 37d93ad..da2cde0 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/FeedClient.java
@@ -20,6 +20,7 @@
import java.util.logging.Logger;
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.builders.UnorderedListBuilder;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -29,6 +30,7 @@
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.AMutableOrderedList;
import edu.uci.ics.asterix.om.base.AMutablePoint;
import edu.uci.ics.asterix.om.base.AMutableRecord;
import edu.uci.ics.asterix.om.base.AMutableString;
@@ -36,18 +38,16 @@
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.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;
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 {
+public abstract class FeedClient implements IFeedClient {
- protected static final Logger LOGGER = Logger.getLogger(PullBasedFeedClient.class.getName());
+ protected static final Logger LOGGER = Logger.getLogger(FeedClient.class.getName());
protected ARecordSerializerDeserializer recordSerDe;
protected AMutableRecord mutableRecord;
@@ -70,7 +70,7 @@
protected ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
.getSerializerDeserializer(BuiltinType.AINT32);
- public abstract InflowState setNextRecord() throws Exception;
+ public abstract InflowState retrieveNextRecord() throws Exception;
@Override
public InflowState nextTuple(DataOutput dataOutput, int timeout) throws AsterixException {
@@ -79,12 +79,9 @@
int waitCount = 0;
boolean continueWait = true;
while ((state == null || state.equals(InflowState.DATA_NOT_AVAILABLE)) && continueWait) {
- state = setNextRecord();
+ state = retrieveNextRecord();
switch (state) {
case DATA_AVAILABLE:
- IAType t = mutableRecord.getType();
- ATypeTag tag = t.getTypeTag();
- dataOutput.writeByte(tag.serialize());
recordBuilder.reset(mutableRecord.getType());
recordBuilder.init();
writeRecord(mutableRecord, dataOutput, recordBuilder);
@@ -94,7 +91,7 @@
continueWait = false;
} else {
if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Waiting to obtaing data from pull based adapter");
+ LOGGER.warning("Waiting to obtain data from pull based adaptor");
}
Thread.sleep(1000);
waitCount++;
@@ -121,30 +118,35 @@
writeObject(obj, fieldValue.getDataOutput());
recordBuilder.addField(pos, fieldValue);
}
- recordBuilder.write(dataOutput, false);
+ recordBuilder.write(dataOutput, true);
}
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);
- }
+ case RECORD: {
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);
+ }
+
+ case ORDEREDLIST: {
+ OrderedListBuilder listBuilder = new OrderedListBuilder();
+ listBuilder.reset((AOrderedListType) ((AMutableOrderedList) obj).getType());
+ IACursor cursor = ((AMutableOrderedList) 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, true);
+ break;
+ }
+
+ case UNORDEREDLIST: {
UnorderedListBuilder listBuilder = new UnorderedListBuilder();
listBuilder.reset((AUnorderedListType) ((AMutableUnorderedList) obj).getType());
IACursor cursor = ((AMutableUnorderedList) obj).getCursor();
@@ -155,8 +157,10 @@
writeObject(item, listItemValue.getDataOutput());
listBuilder.addItem(listItemValue);
}
- listBuilder.write(dataOutput, false);
+ listBuilder.write(dataOutput, true);
break;
+ }
+
default:
AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(obj.getType()).serialize(obj,
dataOutput);
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 c2a8a95..f9793f6 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
@@ -17,9 +17,8 @@
import java.io.IOException;
import java.io.InputStream;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.runtime.operators.file.AbstractTupleParser;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -34,23 +33,22 @@
public abstract InputStream getInputStream(int partition) throws IOException;
- protected final ITupleParser tupleParser;
+ protected final ITupleParserFactory parserFactory;
+ protected 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.parserFactory = parserFactory;
this.sourceDatatype = sourceDatatype;
this.ctx = ctx;
}
@Override
public void start(int partition, IFrameWriter writer) throws Exception {
+ tupleParser = parserFactory.createTupleParser(ctx);
InputStream in = getInputStream(partition);
- if (tupleParser instanceof AbstractTupleParser) {
- ((AbstractTupleParser) tupleParser).setFilename(getFilename(partition));
- }
tupleParser.parse(in, writer);
}
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 9f4b97c..8036a7b 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
@@ -23,6 +23,7 @@
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.SequenceFileInputFormat;
import org.apache.hadoop.mapred.TextInputFormat;
+
import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
import edu.uci.ics.asterix.external.indexing.input.GenericFileAwareRecordReader;
import edu.uci.ics.asterix.external.indexing.input.GenericRecordReader;
@@ -30,6 +31,7 @@
import edu.uci.ics.asterix.external.indexing.input.TextualFullScanDataReader;
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory;
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;
@@ -71,9 +73,9 @@
@Override
public InputStream getInputStream(int partition) throws IOException {
if ((conf.getInputFormat() instanceof TextInputFormat || conf.getInputFormat() instanceof SequenceFileInputFormat)
- && (HDFSAdapterFactory.FORMAT_ADM.equalsIgnoreCase((String) configuration
- .get(HDFSAdapterFactory.KEY_FORMAT)) || HDFSAdapterFactory.FORMAT_DELIMITED_TEXT
- .equalsIgnoreCase((String) configuration.get(HDFSAdapterFactory.KEY_FORMAT)))) {
+ && (AsterixTupleParserFactory.FORMAT_ADM.equalsIgnoreCase((String) configuration
+ .get(AsterixTupleParserFactory.KEY_FORMAT)) || AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT
+ .equalsIgnoreCase((String) configuration.get(AsterixTupleParserFactory.KEY_FORMAT)))) {
if (files != null) {
return new TextualDataReader(inputSplits, readSchedule, nodeName, conf, executed, files);
} else {
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
index 9af1dd7..f730f28 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
@@ -27,6 +27,7 @@
import edu.uci.ics.asterix.external.indexing.input.TextualDataReader;
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
@@ -64,8 +65,8 @@
public InputStream getInputStream(int partition) throws IOException {
if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_RC)) {
return new RCFileDataReader(inputSplits, readSchedule, nodeName, conf, executed, files);
- } else if (format.equals(HDFSAdapterFactory.FORMAT_ADM)
- || format.equals(HDFSAdapterFactory.FORMAT_DELIMITED_TEXT)) {
+ } else if (format.equals(AsterixTupleParserFactory.FORMAT_ADM)
+ || format.equals(AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT)) {
return new TextualDataReader(inputSplits, readSchedule, nodeName, conf, executed, files);
} else {
return new GenericFileAwareRecordReader(inputSplits, readSchedule, nodeName, conf, executed, files);
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IFeedClient.java
new file mode 100644
index 0000000..02e60b4
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IFeedClient.java
@@ -0,0 +1,42 @@
+/*
+ * 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.DataOutput;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+
+public interface IFeedClient {
+
+ public enum InflowState {
+ NO_MORE_DATA,
+ DATA_AVAILABLE,
+ DATA_NOT_AVAILABLE
+ }
+
+ /**
+ * Writes the next fetched tuple into the provided instance of DatatOutput. Invocation of this method blocks until
+ * a new tuple has been written or the specified time has expired.
+ *
+ * @param dataOutput
+ * The receiving channel for the feed client to write ADM records to.
+ * @param timeout
+ * Threshold time (expressed in seconds) for the next tuple to be obtained from the external source.
+ * @return
+ * @throws AsterixException
+ */
+ public InflowState nextTuple(DataOutput dataOutput, int timeout) throws AsterixException;
+
+}
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 78554f2..b31e824 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
@@ -21,7 +21,6 @@
import java.io.InputStream;
import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.runtime.operators.file.AbstractTupleParser;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -58,15 +57,7 @@
}
}
- @Override
- public void start(int partition, IFrameWriter writer) throws Exception {
- InputStream in = getInputStream(partition);
- if (tupleParser instanceof AbstractTupleParser) {
- ((AbstractTupleParser) tupleParser).setFilename(getFilename(partition));
- }
- tupleParser.parse(in, writer);
- }
-
+
@Override
public String getFilename(int partition) {
final FileSplit fileSplit = fileSplits[partition];
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAzureTwitterAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAzureTwitterAdapter.java
index c739ca3..63181dc 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAzureTwitterAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PullBasedAzureTwitterAdapter.java
@@ -9,7 +9,7 @@
import com.microsoft.windowsazure.services.core.storage.CloudStorageAccount;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -77,4 +77,9 @@
public DataExchangeMode getDataExchangeMode() {
return DataExchangeMode.PULL;
}
+
+ @Override
+ public boolean handleException(Exception e) {
+ return false;
+ }
}
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 838cfeb..27d7049 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
@@ -17,30 +17,33 @@
import java.util.Map;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IFeedAdapter;
+import edu.uci.ics.asterix.common.parse.ITupleForwardPolicy;
import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.CounterTimerTupleForwardPolicy;
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 IFeedAdapter {
+public class PullBasedTwitterAdapter extends ClientBasedFeedAdapter implements IFeedAdapter {
private static final long serialVersionUID = 1L;
- public static final String QUERY = "query";
- public static final String INTERVAL = "interval";
+ private static final int DEFAULT_BATCH_SIZE = 5;
private ARecordType recordType;
private PullBasedTwitterFeedClient tweetClient;
@Override
- public IPullBasedFeedClient getFeedClient(int partition) {
+ public IFeedClient getFeedClient(int partition) {
return tweetClient;
}
- public PullBasedTwitterAdapter(Map<String, String> configuration, ARecordType recordType, IHyracksTaskContext ctx) throws AsterixException {
+ public PullBasedTwitterAdapter(Map<String, String> configuration, ARecordType recordType, IHyracksTaskContext ctx)
+ throws AsterixException {
super(configuration, ctx);
tweetClient = new PullBasedTwitterFeedClient(ctx, recordType, this);
}
@@ -54,4 +57,20 @@
return DataExchangeMode.PULL;
}
+ @Override
+ public boolean handleException(Exception e) {
+ return true;
+ }
+
+ @Override
+ public ITupleForwardPolicy getTupleParserPolicy() {
+ configuration.put(ITupleForwardPolicy.PARSER_POLICY,
+ ITupleForwardPolicy.TupleForwardPolicyType.COUNTER_TIMER_EXPIRED.name());
+ String propValue = configuration.get(CounterTimerTupleForwardPolicy.BATCH_SIZE);
+ if (propValue == null) {
+ configuration.put(CounterTimerTupleForwardPolicy.BATCH_SIZE, "" + DEFAULT_BATCH_SIZE);
+ }
+ return AsterixTupleParserFactory.getTupleParserPolicy(configuration);
+ }
+
}
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 2c8d659..2b68c88 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
@@ -16,93 +16,86 @@
import java.util.List;
import java.util.Map;
-import java.util.UUID;
import twitter4j.Query;
import twitter4j.QueryResult;
-import twitter4j.Tweet;
+import twitter4j.Status;
import twitter4j.Twitter;
import twitter4j.TwitterException;
-import twitter4j.TwitterFactory;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
-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.external.util.TweetProcessor;
+import edu.uci.ics.asterix.external.util.TwitterUtil;
+import edu.uci.ics.asterix.external.util.TwitterUtil.SearchAPIConstants;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
/**
- * An implementation of @see {PullBasedFeedClient} for the Twitter service.
- * The feed client fetches data from Twitter service by sending request at
- * regular (configurable) interval.
+ * An implementation of @see {PullBasedFeedClient} for the Twitter service. The
+ * feed client fetches data from Twitter service by sending request at regular
+ * (configurable) interval.
*/
-public class PullBasedTwitterFeedClient extends PullBasedFeedClient {
+public class PullBasedTwitterFeedClient extends FeedClient {
private String keywords;
private Query query;
private Twitter twitter;
- private int requestInterval = 10; // seconds
+ private int requestInterval = 5; // seconds
private QueryResult result;
- private IAObject[] mutableFields;
- private String[] tupleFieldValues;
private ARecordType recordType;
private int nextTweetIndex = 0;
+ private long lastTweetIdReceived = 0;
+ private TweetProcessor tweetProcessor;
public PullBasedTwitterFeedClient(IHyracksTaskContext ctx, ARecordType recordType, PullBasedTwitterAdapter adapter) {
- twitter = new TwitterFactory().getInstance();
- mutableFields = new IAObject[] { new AMutableString(null), new AMutableString(null), new AMutableString(null),
- new AMutableString(null), new AMutableString(null) };
+ this.twitter = TwitterUtil.getTwitterService(adapter.getConfiguration());
this.recordType = recordType;
- recordSerDe = new ARecordSerializerDeserializer(recordType);
- mutableRecord = new AMutableRecord(recordType, mutableFields);
- tupleFieldValues = new String[recordType.getFieldNames().length];
- initialize(adapter.getConfiguration());
+ this.tweetProcessor = new TweetProcessor(recordType);
+ this.recordSerDe = new ARecordSerializerDeserializer(recordType);
+ this.mutableRecord = tweetProcessor.getMutableRecord();
+ this.initialize(adapter.getConfiguration());
}
public ARecordType getRecordType() {
return recordType;
}
- public AMutableRecord getMutableRecord() {
- return mutableRecord;
- }
-
@Override
- public InflowState setNextRecord() throws Exception {
- Tweet tweet;
+ public InflowState retrieveNextRecord() throws Exception {
+ Status tweet;
tweet = getNextTweet();
if (tweet == null) {
return InflowState.DATA_NOT_AVAILABLE;
}
- int numFields = recordType.getFieldNames().length;
- tupleFieldValues[0] = UUID.randomUUID().toString();
- tupleFieldValues[1] = tweet.getFromUser();
- tupleFieldValues[2] = tweet.getLocation() == null ? "" : tweet.getLocation();
- tupleFieldValues[3] = tweet.getText();
- tupleFieldValues[4] = tweet.getCreatedAt().toString();
- for (int i = 0; i < numFields; i++) {
- ((AMutableString) mutableFields[i]).setValue(tupleFieldValues[i]);
- mutableRecord.setValueAtPos(i, mutableFields[i]);
- }
+
+ tweetProcessor.processNextTweet(tweet);
return InflowState.DATA_AVAILABLE;
}
private void initialize(Map<String, String> params) {
- this.keywords = (String) params.get(PullBasedTwitterAdapter.QUERY);
- this.requestInterval = Integer.parseInt((String) params.get(PullBasedTwitterAdapter.INTERVAL));
+ this.keywords = (String) params.get(SearchAPIConstants.QUERY);
+ this.requestInterval = Integer.parseInt((String) params.get(SearchAPIConstants.INTERVAL));
this.query = new Query(keywords);
- query.setRpp(100);
+ this.query.setCount(100);
}
- private Tweet getNextTweet() throws TwitterException, InterruptedException {
+ private Status getNextTweet() throws TwitterException, InterruptedException {
if (result == null || nextTweetIndex >= result.getTweets().size()) {
Thread.sleep(1000 * requestInterval);
+ query.setSinceId(lastTweetIdReceived);
result = twitter.search(query);
nextTweetIndex = 0;
}
- List<Tweet> tw = result.getTweets();
- return tw.get(nextTweetIndex++);
+ if (result != null && !result.getTweets().isEmpty()) {
+ List<Status> tw = result.getTweets();
+ Status tweet = tw.get(nextTweetIndex++);
+ if (lastTweetIdReceived < tweet.getId()) {
+ lastTweetIdReceived = tweet.getId();
+ }
+ return tweet;
+ } else {
+ return null;
+ }
}
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PushBasedTwitterAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PushBasedTwitterAdapter.java
new file mode 100644
index 0000000..8bc9a37
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PushBasedTwitterAdapter.java
@@ -0,0 +1,52 @@
+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.common.parse.ITupleForwardPolicy;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.CounterTimerTupleForwardPolicy;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public class PushBasedTwitterAdapter extends ClientBasedFeedAdapter {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final int DEFAULT_BATCH_SIZE = 50;
+
+ private PushBasedTwitterFeedClient tweetClient;
+
+ public PushBasedTwitterAdapter(Map<String, String> configuration, ARecordType recordType, IHyracksTaskContext ctx) throws AsterixException {
+ super(configuration, ctx);
+ this.configuration = configuration;
+ this.tweetClient = new PushBasedTwitterFeedClient(ctx, recordType, this);
+ }
+
+ @Override
+ public DataExchangeMode getDataExchangeMode() {
+ return DataExchangeMode.PUSH;
+ }
+
+ @Override
+ public boolean handleException(Exception e) {
+ return true;
+ }
+
+ @Override
+ public IFeedClient getFeedClient(int partition) throws Exception {
+ return tweetClient;
+ }
+
+ @Override
+ public ITupleForwardPolicy getTupleParserPolicy() {
+ configuration.put(ITupleForwardPolicy.PARSER_POLICY,
+ ITupleForwardPolicy.TupleForwardPolicyType.COUNTER_TIMER_EXPIRED.name());
+ String propValue = configuration.get(CounterTimerTupleForwardPolicy.BATCH_SIZE);
+ if (propValue == null) {
+ configuration.put(CounterTimerTupleForwardPolicy.BATCH_SIZE, "" + DEFAULT_BATCH_SIZE);
+ }
+ return AsterixTupleParserFactory.getTupleParserPolicy(configuration);
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PushBasedTwitterFeedClient.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PushBasedTwitterFeedClient.java
new file mode 100644
index 0000000..908fd34
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/PushBasedTwitterFeedClient.java
@@ -0,0 +1,118 @@
+/*
+ * 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.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import twitter4j.FilterQuery;
+import twitter4j.Query;
+import twitter4j.StallWarning;
+import twitter4j.Status;
+import twitter4j.StatusDeletionNotice;
+import twitter4j.StatusListener;
+import twitter4j.TwitterStream;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import edu.uci.ics.asterix.external.util.TweetProcessor;
+import edu.uci.ics.asterix.external.util.TwitterUtil;
+import edu.uci.ics.asterix.external.util.TwitterUtil.SearchAPIConstants;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * An implementation of @see {PullBasedFeedClient} for the Twitter service. The
+ * feed client fetches data from Twitter service by sending request at regular
+ * (configurable) interval.
+ */
+public class PushBasedTwitterFeedClient extends FeedClient {
+
+ private String keywords;
+ private Query query;
+
+ private ARecordType recordType;
+ private TweetProcessor tweetProcessor;
+ private LinkedBlockingQueue<Status> inputQ;
+
+ public PushBasedTwitterFeedClient(IHyracksTaskContext ctx, ARecordType recordType, PushBasedTwitterAdapter adapter) throws AsterixException {
+ this.recordType = recordType;
+ this.tweetProcessor = new TweetProcessor(recordType);
+ this.recordSerDe = new ARecordSerializerDeserializer(recordType);
+ this.mutableRecord = tweetProcessor.getMutableRecord();
+ this.initialize(adapter.getConfiguration());
+ this.inputQ = new LinkedBlockingQueue<Status>();
+ TwitterStream twitterStream = TwitterUtil.getTwitterStream(adapter.getConfiguration());
+ twitterStream.addListener(new TweetListener(inputQ));
+ FilterQuery query = TwitterUtil.getFilterQuery(adapter.getConfiguration());
+ if (query != null) {
+ twitterStream.filter(query);
+ } else {
+ twitterStream.sample();
+ }
+ }
+
+ public ARecordType getRecordType() {
+ return recordType;
+ }
+
+ private class TweetListener implements StatusListener {
+
+ private LinkedBlockingQueue<Status> inputQ;
+
+ public TweetListener(LinkedBlockingQueue<Status> inputQ) {
+ this.inputQ = inputQ;
+ }
+
+ @Override
+ public void onStatus(Status tweet) {
+ inputQ.add(tweet);
+ }
+
+ @Override
+ public void onException(Exception arg0) {
+
+ }
+
+ @Override
+ public void onDeletionNotice(StatusDeletionNotice arg0) {
+ }
+
+ @Override
+ public void onScrubGeo(long arg0, long arg1) {
+ }
+
+ @Override
+ public void onStallWarning(StallWarning arg0) {
+ }
+
+ @Override
+ public void onTrackLimitationNotice(int arg0) {
+ }
+ }
+
+ @Override
+ public InflowState retrieveNextRecord() throws Exception {
+ Status tweet = inputQ.take();
+ tweetProcessor.processNextTweet(tweet);
+ return InflowState.DATA_AVAILABLE;
+ }
+
+ private void initialize(Map<String, String> params) {
+ this.keywords = (String) params.get(SearchAPIConstants.QUERY);
+ this.query = new Query(keywords);
+ this.query.setCount(100);
+ }
+
+}
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 4eea034..bcf809d 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,14 +19,16 @@
import java.util.Map;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IFeedAdapter;
+import edu.uci.ics.asterix.common.parse.ITupleForwardPolicy;
import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
/**
* RSSFeedAdapter provides the functionality of fetching an RSS based feed.
*/
-public class RSSFeedAdapter extends PullBasedAdapter implements IFeedAdapter {
+public class RSSFeedAdapter extends ClientBasedFeedAdapter implements IFeedAdapter {
private static final long serialVersionUID = 1L;
@@ -35,7 +37,7 @@
private List<String> feedURLs = new ArrayList<String>();
private String id_prefix = "";
- private IPullBasedFeedClient rssFeedClient;
+ private IFeedClient rssFeedClient;
private ARecordType recordType;
@@ -62,7 +64,7 @@
}
@Override
- public IPullBasedFeedClient getFeedClient(int partition) throws Exception {
+ public IFeedClient getFeedClient(int partition) throws Exception {
if (rssFeedClient == null) {
rssFeedClient = new RSSFeedClient(this, feedURLs.get(partition), id_prefix);
}
@@ -78,4 +80,14 @@
return DataExchangeMode.PULL;
}
+ @Override
+ public boolean handleException(Exception e) {
+ return false;
+ }
+
+ @Override
+ public ITupleForwardPolicy getTupleParserPolicy() {
+ return AsterixTupleParserFactory.getTupleParserPolicy(configuration);
+ }
+
}
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 41ed923..e7cbd16 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
@@ -39,7 +39,7 @@
* fetching from an RSS feed source at regular interval.
*/
@SuppressWarnings("rawtypes")
-public class RSSFeedClient extends PullBasedFeedClient {
+public class RSSFeedClient extends FeedClient {
private long id = 0;
private String idPrefix;
@@ -79,7 +79,7 @@
}
@Override
- public InflowState setNextRecord() throws Exception {
+ public InflowState retrieveNextRecord() throws Exception {
SyndEntryImpl feedEntry = getNextRSSFeed();
if (feedEntry == null) {
return InflowState.DATA_NOT_AVAILABLE;
@@ -133,9 +133,9 @@
class FetcherEventListenerImpl implements FetcherListener {
- private final IPullBasedFeedClient feedClient;
+ private final IFeedClient feedClient;
- public FetcherEventListenerImpl(IPullBasedFeedClient feedClient) {
+ public FetcherEventListenerImpl(IFeedClient feedClient) {
this.feedClient = feedClient;
}
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
index 80965b0..25a0221 100644
--- 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
@@ -5,7 +5,7 @@
import java.util.logging.Level;
import java.util.logging.Logger;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.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;
@@ -25,7 +25,7 @@
protected final IAType sourceDatatype;
- public StreamBasedAdapter(ITupleParserFactory parserFactory, IAType sourceDatatype, IHyracksTaskContext ctx)
+ public StreamBasedAdapter(ITupleParserFactory parserFactory, IAType sourceDatatype, IHyracksTaskContext ctx, int partition)
throws HyracksDataException {
this.tupleParser = parserFactory.createTupleParser(ctx);
this.sourceDatatype = sourceDatatype;
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSIndexingParserFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSIndexingParserFactory.java
index e2ef9fd..ca15b359 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSIndexingParserFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSIndexingParserFactory.java
@@ -24,6 +24,7 @@
import edu.uci.ics.asterix.external.adapter.factory.StreamBasedAdapterFactory;
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.AsterixTupleParserFactory;
import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -84,11 +85,11 @@
* 2. RC indexing tuple parser
* 3. textual data tuple parser
*/
- if (format.equalsIgnoreCase(StreamBasedAdapterFactory.FORMAT_ADM)) {
+ if (format.equalsIgnoreCase(AsterixTupleParserFactory.FORMAT_ADM)) {
// choice 3 with adm data parser
ADMDataParser dataParser = new ADMDataParser();
return new AdmOrDelimitedIndexingTupleParser(ctx, atype, dataParser);
- } else if (format.equalsIgnoreCase(StreamBasedAdapterFactory.FORMAT_DELIMITED_TEXT)) {
+ } else if (format.equalsIgnoreCase(AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT)) {
// choice 3 with delimited data parser
DelimitedDataParser dataParser = HDFSIndexingAdapterFactory.getDelimitedDataParser(atype,
delimiter, quote);
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSLookupAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSLookupAdapter.java
index c6176fc..7e0d9f9 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSLookupAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/indexing/dataflow/HDFSLookupAdapter.java
@@ -35,6 +35,7 @@
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.IAType;
import edu.uci.ics.asterix.runtime.operators.file.ADMDataParser;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory;
import edu.uci.ics.asterix.runtime.operators.file.DelimitedDataParser;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -81,7 +82,7 @@
// Create the lookup reader and the controlled parser
if (configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_RC)) {
configureRCFile(jobConf, iNullWriterFactory);
- } else if (configuration.get(HDFSAdapterFactory.KEY_FORMAT).equals(HDFSAdapterFactory.FORMAT_ADM)) {
+ } else if (configuration.get(AsterixTupleParserFactory.KEY_FORMAT).equals(AsterixTupleParserFactory.FORMAT_ADM)) {
// create an adm parser
ADMDataParser dataParser = new ADMDataParser();
if (configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT)) {
@@ -95,10 +96,10 @@
parser = new AdmOrDelimitedControlledTupleParser(ctx, (ARecordType) atype, in, propagateInput,
inRecDesc, dataParser, propagatedFields, ridFields, retainNull, iNullWriterFactory);
}
- } else if (configuration.get(HDFSAdapterFactory.KEY_FORMAT).equals(HDFSAdapterFactory.FORMAT_DELIMITED_TEXT)) {
+ } else if (configuration.get(AsterixTupleParserFactory.KEY_FORMAT).equals(AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT)) {
// create a delimited text parser
- char delimiter = StreamBasedAdapterFactory.getDelimiter(configuration);
- char quote = StreamBasedAdapterFactory.getQuote(configuration, delimiter);
+ char delimiter = AsterixTupleParserFactory.getDelimiter(configuration);
+ char quote = AsterixTupleParserFactory.getQuote(configuration, delimiter);
DelimitedDataParser dataParser = HDFSIndexingAdapterFactory.getDelimitedDataParser((ARecordType) atype,
delimiter, quote);
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
index a148e66..57a8e48 100755
--- 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
@@ -90,9 +90,9 @@
castBuffer.reset();
ATypeHierarchy.convertNumericTypeByteArray(inputVal.getByteArray(), inputVal.getStartOffset(),
inputVal.getLength(), targetTypeTag, castBuffer.getDataOutput());
- functionHelper.setArgument(i, castBuffer.getByteArray());
+ functionHelper.setArgument(i, castBuffer);
} else {
- functionHelper.setArgument(i, inputVal.getByteArray());
+ functionHelper.setArgument(i, inputVal);
}
}
}
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
index fc629ea..d989323 100755
--- 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
@@ -14,9 +14,6 @@
*/
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;
@@ -26,19 +23,14 @@
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;
+ return new ExternalScalarFunction(finfo, args, outputProvider);
case AGGREGATE:
case UNNEST:
- throw new IllegalArgumentException(" not supported function kind" + finfo.getKind());
+ throw new IllegalArgumentException(" UDF of kind" + finfo.getKind() + " not supported.");
default:
throw new IllegalArgumentException(" unknown function kind" + finfo.getKind());
}
@@ -62,9 +54,10 @@
try {
setArguments(tuple);
evaluate(functionHelper);
+ functionHelper.reset();
} catch (Exception e) {
e.printStackTrace();
- throw new AlgebricksException(e);
+ //throw new AlgebricksException(e);
}
}
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
index 43eef52..6109588 100755
--- 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
@@ -29,4 +29,6 @@
public void setResult(IJObject result) throws IOException, AsterixException;
public IJObject getObject(JTypeTag jtypeTag);
+
+ public void reset();
}
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
index 3c5ddfd..4beb259b 100644
--- 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
@@ -14,7 +14,6 @@
*/
package edu.uci.ics.asterix.external.library;
-import java.util.ArrayList;
import java.util.List;
import edu.uci.ics.asterix.external.library.java.IJObject;
@@ -48,6 +47,11 @@
public class JTypeObjectFactory implements IObjectFactory<IJObject, IAType> {
+ public static final JTypeObjectFactory INSTANCE = new JTypeObjectFactory();
+
+ private JTypeObjectFactory() {
+ }
+
@Override
public IJObject create(IAType type) {
IJObject retValue = null;
@@ -77,7 +81,7 @@
retValue = new JPoint3D(0, 0, 0);
break;
case POLYGON:
- retValue = new JPolygon(new ArrayList<JPoint>());
+ retValue = new JPolygon(new JPoint[] {});
break;
case LINE:
retValue = new JLine(new JPoint(0, 0), new JPoint(0, 0));
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
index 192cf3e..7f5b3bc 100644
--- 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
@@ -14,59 +14,55 @@
*/
package edu.uci.ics.asterix.external.library;
-import java.io.DataOutput;
import java.io.IOException;
-import java.util.List;
+import java.util.HashMap;
+import java.util.Map;
-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.JObjectPointableVisitor;
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.pointables.AFlatValuePointable;
+import edu.uci.ics.asterix.om.pointables.AListPointable;
+import edu.uci.ics.asterix.om.pointables.ARecordPointable;
+import edu.uci.ics.asterix.om.pointables.PointableAllocator;
+import edu.uci.ics.asterix.om.pointables.base.IVisitablePointable;
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;
+import edu.uci.ics.hyracks.data.std.api.IValueReference;
public class JavaFunctionHelper implements IFunctionHelper {
private final IExternalFunctionInfo finfo;
private final IDataOutputProvider outputProvider;
- private IJObject[] arguments;
+ private final IJObject[] arguments;
private IJObject resultHolder;
- 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);
+ private final IObjectPool<IJObject, IAType> objectPool = new ListObjectPool<IJObject, IAType>(
+ JTypeObjectFactory.INSTANCE);
+ private final JObjectPointableVisitor pointableVisitor;
+ private final PointableAllocator pointableAllocator;
+ private final Map<Integer, TypeInfo> poolTypeInfo;
public JavaFunctionHelper(IExternalFunctionInfo finfo, IDataOutputProvider outputProvider)
throws AlgebricksException {
this.finfo = finfo;
this.outputProvider = outputProvider;
- List<IAType> params = finfo.getParamList();
- arguments = new IJObject[params.size()];
+ this.pointableVisitor = new JObjectPointableVisitor();
+ this.pointableAllocator = new PointableAllocator();
+ this.arguments = new IJObject[finfo.getParamList().size()];
int index = 0;
- for (IAType param : params) {
- this.arguments[index] = objectPool.allocate(param);
- index++;
+ for (IAType param : finfo.getParamList()) {
+ this.arguments[index++] = objectPool.allocate(param);
}
- resultHolder = objectPool.allocate(finfo.getReturnType());
+ this.resultHolder = objectPool.allocate(finfo.getReturnType());
+ this.poolTypeInfo = new HashMap<Integer, TypeInfo>();
+
}
@Override
@@ -76,110 +72,55 @@
@Override
public void setResult(IJObject result) throws IOException, AsterixException {
- IAObject obj = result.getIAObject();
try {
- outputProvider.getDataOutput().writeByte(obj.getType().getTypeTag().serialize());
- } catch (IOException e) {
+ result.serialize(outputProvider.getDataOutput(), true);
+ result.reset();
+ } catch (IOException | AlgebricksException 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()) {
+ public void setArgument(int index, IValueReference valueReference) throws IOException, AsterixException {
+ IVisitablePointable pointable = null;
+ IJObject jObject = null;
+ IAType type = finfo.getParamList().get(index);
+ switch (type.getTypeTag()) {
case RECORD:
- reset((JRecord) resultHolder);
+ pointable = pointableAllocator.allocateRecordValue(type);
+ pointable.set(valueReference);
+ jObject = pointableVisitor.visit((ARecordPointable) pointable, getTypeInfo(index, type));
+ break;
+ case ORDEREDLIST:
+ case UNORDEREDLIST:
+ pointable = pointableAllocator.allocateListValue(type);
+ pointable.set(valueReference);
+ jObject = pointableVisitor.visit((AListPointable) pointable, getTypeInfo(index, type));
+ break;
+ case ANY:
+ throw new IllegalStateException("Cannot handle a function argument of type " + type.getTypeTag());
+ default:
+ pointable = pointableAllocator.allocateFieldValue(type);
+ pointable.set(valueReference);
+ jObject = pointableVisitor.visit((AFlatValuePointable) pointable, getTypeInfo(index, type));
break;
}
+ arguments[index] = jObject;
}
- private void reset(JRecord jRecord) {
- List<IJObject> fields = ((JRecord) jRecord).getFields();
- for (IJObject field : fields) {
- switch (field.getTypeTag()) {
- case RECORD:
- reset((JRecord) field);
- break;
- }
+ private TypeInfo getTypeInfo(int index, IAType type) {
+ TypeInfo typeInfo = poolTypeInfo.get(index);
+ if (typeInfo == null) {
+ typeInfo = new TypeInfo(objectPool, type, type.getTypeTag());
+ poolTypeInfo.put(index, typeInfo);
}
- 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);
+ return typeInfo;
}
@Override
public IJObject getResultObject() {
+ if (resultHolder == null) {
+ resultHolder = objectPool.allocate(finfo.getReturnType());
+ }
return resultHolder;
}
@@ -197,4 +138,10 @@
return retValue;
}
+ @Override
+ public void reset() {
+ pointableAllocator.reset();
+ objectPool.reset();
+ }
+
}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/TypeInfo.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/TypeInfo.java
new file mode 100644
index 0000000..e2c66ca
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/TypeInfo.java
@@ -0,0 +1,49 @@
+package edu.uci.ics.asterix.external.library;
+
+import edu.uci.ics.asterix.external.library.java.IJObject;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.container.IObjectPool;
+
+public class TypeInfo {
+
+ private IObjectPool<IJObject, IAType> objectPool;
+ private IAType atype;
+ private ATypeTag typeTag;
+
+ public TypeInfo(IObjectPool<IJObject, IAType> objectPool, IAType atype, ATypeTag typeTag) {
+ this.objectPool = objectPool;
+ this.atype = atype;
+ this.typeTag = typeTag;
+ }
+
+ public void reset(IAType atype, ATypeTag typeTag) {
+ this.atype = atype;
+ this.typeTag = typeTag;
+ }
+
+ public IObjectPool<IJObject, IAType> getObjectPool() {
+ return objectPool;
+ }
+
+ public void setObjectPool(IObjectPool<IJObject, IAType> objectPool) {
+ this.objectPool = objectPool;
+ }
+
+ public IAType getAtype() {
+ return atype;
+ }
+
+ public void setAtype(IAType atype) {
+ this.atype = atype;
+ }
+
+ public ATypeTag getTypeTag() {
+ return typeTag;
+ }
+
+ public void setTypeTag(ATypeTag typeTag) {
+ this.typeTag = typeTag;
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJListAccessor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJListAccessor.java
new file mode 100644
index 0000000..87db84a
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJListAccessor.java
@@ -0,0 +1,12 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import edu.uci.ics.asterix.om.pointables.AListPointable;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.container.IObjectPool;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IJListAccessor {
+
+ IJObject access(AListPointable pointable, IObjectPool<IJObject, IAType> objectPool, IAType listType,
+ JObjectPointableVisitor pointableVisitor) throws HyracksDataException;
+}
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
index ff8e563..3567e7f 100644
--- 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
@@ -14,12 +14,20 @@
*/
package edu.uci.ics.asterix.external.library.java;
+import java.io.DataOutput;
+
import edu.uci.ics.asterix.om.base.IAObject;
import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public interface IJObject {
public ATypeTag getTypeTag();
public IAObject getIAObject();
+
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException;
+
+ public void reset() throws AlgebricksException;
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJObjectAccessor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJObjectAccessor.java
new file mode 100644
index 0000000..6967243
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJObjectAccessor.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import edu.uci.ics.asterix.om.pointables.base.IVisitablePointable;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.container.IObjectPool;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IJObjectAccessor {
+ IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> obj) throws HyracksDataException;
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJRecordAccessor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJRecordAccessor.java
new file mode 100644
index 0000000..55ae262
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/IJRecordAccessor.java
@@ -0,0 +1,15 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import edu.uci.ics.asterix.external.library.java.JObjects.JRecord;
+import edu.uci.ics.asterix.om.pointables.ARecordPointable;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.container.IObjectPool;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IJRecordAccessor {
+
+ public JRecord access(ARecordPointable pointable, IObjectPool<IJObject, IAType> objectPool, ARecordType recordType,
+ JObjectPointableVisitor pointableVisitor) throws HyracksDataException;
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectAccessors.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectAccessors.java
new file mode 100644
index 0000000..02d11b9
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectAccessors.java
@@ -0,0 +1,571 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.UnsupportedEncodingException;
+import java.util.LinkedHashMap;
+import java.util.List;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ACircleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ALineSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.APoint3DSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
+import edu.uci.ics.asterix.external.library.TypeInfo;
+import edu.uci.ics.asterix.external.library.java.JObjects.JBoolean;
+import edu.uci.ics.asterix.external.library.java.JObjects.JByte;
+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.JList;
+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.external.util.TweetProcessor;
+import edu.uci.ics.asterix.om.base.ACircle;
+import edu.uci.ics.asterix.om.base.ADuration;
+import edu.uci.ics.asterix.om.base.ALine;
+import edu.uci.ics.asterix.om.base.APoint;
+import edu.uci.ics.asterix.om.base.APoint3D;
+import edu.uci.ics.asterix.om.base.APolygon;
+import edu.uci.ics.asterix.om.base.ARectangle;
+import edu.uci.ics.asterix.om.pointables.AFlatValuePointable;
+import edu.uci.ics.asterix.om.pointables.AListPointable;
+import edu.uci.ics.asterix.om.pointables.ARecordPointable;
+import edu.uci.ics.asterix.om.pointables.base.IVisitablePointable;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AbstractCollectionType;
+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.container.IObjectPool;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class JObjectAccessors {
+
+ public static IJObjectAccessor createFlatJObjectAccessor(ATypeTag aTypeTag) {
+ IJObjectAccessor accessor = null;
+ switch (aTypeTag) {
+ case BOOLEAN:
+ accessor = new JBooleanAccessor();
+ break;
+ case INT8:
+ accessor = new JInt8Accessor();
+ break;
+ case INT16:
+ accessor = new JInt16Accessor();
+ break;
+ case INT32:
+ accessor = new JInt32Accessor();
+ break;
+ case INT64:
+ accessor = new JInt64Accessor();
+ break;
+ case FLOAT:
+ accessor = new JFloatAccessor();
+ break;
+ case DOUBLE:
+ accessor = new JDoubleAccessor();
+ break;
+ case STRING:
+ accessor = new JStringAccessor();
+ break;
+ case POINT:
+ accessor = new JPointAccessor();
+ break;
+ case POINT3D:
+ accessor = new JPoint3DAccessor();
+ break;
+ case LINE:
+ accessor = new JLineAccessor();
+ break;
+ case DATE:
+ accessor = new JDateAccessor();
+ break;
+ case DATETIME:
+ accessor = new JDateTimeAccessor();
+ break;
+ case DURATION:
+ accessor = new JDurationAccessor();
+ break;
+ }
+ return accessor;
+ }
+
+ public static class JInt8Accessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ byte o = AInt8SerializerDeserializer.getByte(b, s + 1);
+ IJObject jObject = objectPool.allocate(BuiltinType.AINT8);
+ ((JByte) jObject).setValue(o);
+ return null;
+ }
+
+ }
+
+ public static class JInt16Accessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ short i = AInt16SerializerDeserializer.getShort(b, s + 1);
+ IJObject jObject = objectPool.allocate(BuiltinType.AINT16);
+ ((JInt) jObject).setValue(i);
+ return null;
+ }
+ }
+
+ public static class JInt32Accessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ int i = AInt32SerializerDeserializer.getInt(b, s + 1);
+ IJObject jObject = objectPool.allocate(BuiltinType.AINT32);
+ ((JInt) jObject).setValue(i);
+ return jObject;
+ }
+ }
+
+ public static class JInt64Accessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ long v = AInt64SerializerDeserializer.getLong(b, s + 1);
+ IJObject jObject = objectPool.allocate(BuiltinType.AINT64);
+ ((JLong) jObject).setValue(v);
+ return jObject;
+ }
+ }
+
+ public static class JFloatAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ float v = AFloatSerializerDeserializer.getFloat(b, s + 1);
+ IJObject jObject = objectPool.allocate(BuiltinType.AFLOAT);
+ ((JFloat) jObject).setValue(v);
+ return jObject;
+ }
+ }
+
+ public static class JDoubleAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ double v = ADoubleSerializerDeserializer.getDouble(b, s + 1);
+ IJObject jObject = objectPool.allocate(BuiltinType.ADOUBLE);
+ ((JDouble) jObject).setValue(v);
+ return jObject;
+ }
+ }
+
+ public static class JStringAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ int l = pointable.getLength();
+
+ String v = null;
+ v = AStringSerializerDeserializer.INSTANCE.deserialize(
+ new DataInputStream(new ByteArrayInputStream(b, s+1, l-1))).getStringValue();
+ //v = new String(b, s+1, l, "UTF-8");
+ TweetProcessor.getNormalizedString(v);
+ IJObject jObject = objectPool.allocate(BuiltinType.ASTRING);
+ ((JString) jObject).setValue(TweetProcessor.getNormalizedString(v));
+ return jObject;
+ }
+ }
+
+ public static class JBooleanAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ Boolean v = ABooleanSerializerDeserializer.getBoolean(b, s);
+ IJObject jObject = objectPool.allocate(BuiltinType.ABOOLEAN);
+ ((JBoolean) jObject).setValue(v);
+ return jObject;
+ }
+ }
+
+ public static class JDateAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ int v = ADateSerializerDeserializer.getChronon(b, s);
+ IJObject jObject = objectPool.allocate(BuiltinType.ADATE);
+ ((JDate) jObject).setValue(v);
+ return jObject;
+ }
+ }
+
+ public static class JDateTimeAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ long v = ADateTimeSerializerDeserializer.getChronon(b, s);
+ IJObject jObject = objectPool.allocate(BuiltinType.ADATETIME);
+ ((JDateTime) jObject).setValue(v);
+ return jObject;
+ }
+ }
+
+ public static class JDurationAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ int l = pointable.getLength();
+ ADuration duration = ADurationSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(
+ new ByteArrayInputStream(b, s, l)));
+ IJObject jObject = objectPool.allocate(BuiltinType.ADURATION);
+ ((JDuration) jObject).setValue(duration.getMonths(), duration.getMilliseconds());
+ return jObject;
+ }
+ }
+
+ public static class JTimeAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ int v = ATimeSerializerDeserializer.getChronon(b, s);
+ IJObject jObject = objectPool.allocate(BuiltinType.ATIME);
+ ((JTime) jObject).setValue(v);
+ return jObject;
+ }
+ }
+
+ public static class JIntervalAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ long intervalStart = AIntervalSerializerDeserializer.getIntervalStart(b, s);
+ long intervalEnd = AIntervalSerializerDeserializer.getIntervalEnd(b, s);
+ byte intervalType = AIntervalSerializerDeserializer.getIntervalTimeType(b, s);
+ IJObject jObject = objectPool.allocate(BuiltinType.AINTERVAL);
+ try {
+ ((JInterval) jObject).setValue(intervalStart, intervalEnd, intervalType);
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ return jObject;
+ }
+ }
+
+ // Spatial Types
+
+ public static class JCircleAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ int l = pointable.getLength();
+ ACircle v = ACircleSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(
+ new ByteArrayInputStream(b, s, l)));
+ JPoint jpoint = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+ jpoint.setValue(v.getP().getX(), v.getP().getY());
+ IJObject jObject = objectPool.allocate(BuiltinType.ACIRCLE);
+ ((JCircle) jObject).setValue(jpoint, v.getRadius());
+ return jObject;
+ }
+ }
+
+ public static class JPointAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ int l = pointable.getLength();
+ APoint v = APointSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(new ByteArrayInputStream(
+ b, s, l)));
+ JPoint jObject = (JPoint) objectPool.allocate(BuiltinType.APOINT);
+ jObject.setValue(v.getX(), v.getY());
+ return jObject;
+ }
+ }
+
+ public static class JPoint3DAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ int l = pointable.getLength();
+ APoint3D v = APoint3DSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(
+ new ByteArrayInputStream(b, s, l)));
+ JPoint3D jObject = (JPoint3D) objectPool.allocate(BuiltinType.APOINT3D);
+ jObject.setValue(v.getX(), v.getY(), v.getZ());
+ return jObject;
+ }
+ }
+
+ public static class JLineAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ int l = pointable.getLength();
+ ALine v = ALineSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(new ByteArrayInputStream(b,
+ s, l)));
+ JLine jObject = (JLine) objectPool.allocate(BuiltinType.ALINE);
+ jObject.setValue(v.getP1(), v.getP2());
+ return jObject;
+ }
+ }
+
+ public static class JPolygonAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ int l = pointable.getLength();
+ APolygon v = APolygonSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(
+ new ByteArrayInputStream(b, s, l)));
+ JPolygon jObject = (JPolygon) objectPool.allocate(BuiltinType.APOLYGON);
+ jObject.setValue(v.getPoints());
+ return jObject;
+ }
+ }
+
+ public static class JRectangleAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ byte[] b = pointable.getByteArray();
+ int s = pointable.getStartOffset();
+ int l = pointable.getLength();
+ ARectangle v = ARectangleSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(
+ new ByteArrayInputStream(b, s, l)));
+ JRectangle jObject = (JRectangle) objectPool.allocate(BuiltinType.ARECTANGLE);
+ jObject.setValue(v.getP1(), v.getP2());
+ return jObject;
+ }
+ }
+
+ public static class JRecordAccessor implements IJRecordAccessor {
+
+ private final TypeInfo typeInfo;
+ private final JRecord jRecord;
+ private final IJObject[] jObjects;
+ private final LinkedHashMap<String, IJObject> openFields;
+
+ public JRecordAccessor(ARecordType recordType, IObjectPool<IJObject, IAType> objectPool) {
+ this.typeInfo = new TypeInfo(objectPool, null, null);
+ this.jObjects = new IJObject[recordType.getFieldNames().length];
+ this.jRecord = new JRecord(recordType, jObjects);
+ this.openFields = new LinkedHashMap<String, IJObject>();
+ }
+
+ @Override
+ public JRecord access(ARecordPointable pointable, IObjectPool<IJObject, IAType> objectPool,
+ ARecordType recordType, JObjectPointableVisitor pointableVisitor) throws HyracksDataException {
+ try {
+ jRecord.reset();
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ ARecordPointable recordPointable = (ARecordPointable) pointable;
+ List<IVisitablePointable> fieldPointables = recordPointable.getFieldValues();
+ List<IVisitablePointable> fieldTypeTags = recordPointable.getFieldTypeTags();
+ List<IVisitablePointable> fieldNames = recordPointable.getFieldNames();
+ int index = 0;
+ boolean closedPart = true;
+ try {
+ IJObject fieldObject = null;
+ for (IVisitablePointable fieldPointable : fieldPointables) {
+ closedPart = index < recordType.getFieldTypes().length;
+ IVisitablePointable tt = fieldTypeTags.get(index);
+ IAType fieldType = closedPart ? recordType.getFieldTypes()[index] : null;
+ ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(tt.getByteArray()[tt
+ .getStartOffset()]);
+ IVisitablePointable fieldName = fieldNames.get(index);
+ typeInfo.reset(fieldType, typeTag);
+ switch (typeTag) {
+ case RECORD:
+ fieldObject = pointableVisitor.visit((ARecordPointable) fieldPointable, typeInfo);
+ break;
+ case ORDEREDLIST:
+ case UNORDEREDLIST:
+ if (fieldPointable instanceof AFlatValuePointable) {
+ // value is null
+ fieldObject = null;
+ } else {
+ fieldObject = pointableVisitor.visit((AListPointable) fieldPointable, typeInfo);
+ }
+ break;
+ case ANY:
+ break;
+ default:
+ fieldObject = pointableVisitor.visit((AFlatValuePointable) fieldPointable, typeInfo);
+ }
+ if (closedPart) {
+ jObjects[index] = fieldObject;
+ } else {
+ byte[] b = fieldName.getByteArray();
+ int s = fieldName.getStartOffset();
+ int l = fieldName.getLength();
+ String v = AStringSerializerDeserializer.INSTANCE.deserialize(
+ new DataInputStream(new ByteArrayInputStream(b, s + 1, l - 1))).getStringValue();
+ openFields.put(v, fieldObject);
+ }
+ index++;
+ fieldObject = null;
+ }
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ }
+ return jRecord;
+ }
+
+ public void reset() throws HyracksDataException {
+ try {
+ jRecord.reset();
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ openFields.clear();
+ }
+
+ }
+
+ public static class JListAccessor implements IJListAccessor {
+
+ private final TypeInfo typeInfo;
+
+ public JListAccessor(IObjectPool<IJObject, IAType> objectPool) {
+ this.typeInfo = new TypeInfo(objectPool, null, null);
+ }
+
+ @Override
+ public IJObject access(AListPointable pointable, IObjectPool<IJObject, IAType> objectPool, IAType listType,
+ JObjectPointableVisitor pointableVisitor) throws HyracksDataException {
+ List<IVisitablePointable> items = pointable.getItems();
+ List<IVisitablePointable> itemTags = pointable.getItemTags();
+ JList list = pointable.ordered() ? new JOrderedList(listType) : new JUnorderedList(listType);
+ IJObject listItem = null;
+ int index = 0;
+ try {
+
+ for (IVisitablePointable itemPointable : items) {
+ IVisitablePointable itemTagPointable = itemTags.get(index);
+ ATypeTag itemTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(itemTagPointable
+ .getByteArray()[itemTagPointable.getStartOffset()]);
+ typeInfo.reset(listType.getType(), listType.getTypeTag());
+ switch (itemTypeTag) {
+ case RECORD:
+ listItem = pointableVisitor.visit((ARecordPointable) itemPointable, typeInfo);
+ break;
+ case UNORDEREDLIST:
+ case ORDEREDLIST:
+ listItem = pointableVisitor.visit((AListPointable) itemPointable, typeInfo);
+ break;
+ case ANY:
+ throw new IllegalArgumentException("Cannot parse list item of type "
+ + listType.getTypeTag());
+ default:
+ typeInfo.reset(((AbstractCollectionType) listType).getItemType(),
+ ((AbstractCollectionType) listType).getTypeTag());
+ listItem = pointableVisitor.visit((AFlatValuePointable) itemPointable, typeInfo);
+
+ }
+ ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+ .deserialize(itemPointable.getByteArray()[itemPointable.getStartOffset()]);
+
+ list.add(listItem);
+ }
+ } catch (AsterixException exception) {
+ throw new HyracksDataException(exception);
+ }
+ return list;
+ }
+ }
+
+ public static class JUnorderedListAccessor implements IJObjectAccessor {
+
+ @Override
+ public IJObject access(IVisitablePointable pointable, IObjectPool<IJObject, IAType> objectPool)
+ throws HyracksDataException {
+ return null;
+ }
+
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectPointableVisitor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectPointableVisitor.java
new file mode 100644
index 0000000..0f4ce58
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/library/java/JObjectPointableVisitor.java
@@ -0,0 +1,75 @@
+package edu.uci.ics.asterix.external.library.java;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.library.TypeInfo;
+import edu.uci.ics.asterix.external.library.java.JObjectAccessors.JListAccessor;
+import edu.uci.ics.asterix.external.library.java.JObjectAccessors.JRecordAccessor;
+import edu.uci.ics.asterix.om.pointables.AFlatValuePointable;
+import edu.uci.ics.asterix.om.pointables.AListPointable;
+import edu.uci.ics.asterix.om.pointables.ARecordPointable;
+import edu.uci.ics.asterix.om.pointables.base.IVisitablePointable;
+import edu.uci.ics.asterix.om.pointables.visitor.IVisitablePointableVisitor;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class JObjectPointableVisitor implements IVisitablePointableVisitor<IJObject, TypeInfo> {
+
+ private final Map<ATypeTag, IJObjectAccessor> flatJObjectAccessors = new HashMap<ATypeTag, IJObjectAccessor>();
+ private final Map<IVisitablePointable, IJRecordAccessor> raccessorToJObject = new HashMap<IVisitablePointable, IJRecordAccessor>();
+ private final Map<IVisitablePointable, IJListAccessor> laccessorToPrinter = new HashMap<IVisitablePointable, IJListAccessor>();
+
+ @Override
+ public IJObject visit(AListPointable accessor, TypeInfo arg) throws AsterixException {
+ IJObject result = null;
+ IJListAccessor jListAccessor = laccessorToPrinter.get(accessor);
+ if (jListAccessor == null) {
+ jListAccessor = new JListAccessor(arg.getObjectPool());
+ laccessorToPrinter.put(accessor, jListAccessor);
+ }
+ try {
+ result = jListAccessor.access(accessor, arg.getObjectPool(), arg.getAtype(), this);
+ } catch (Exception e) {
+ throw new AsterixException(e);
+ }
+ return result;
+ }
+
+ @Override
+ public IJObject visit(ARecordPointable accessor, TypeInfo arg) throws AsterixException {
+ IJObject result = null;
+ IJRecordAccessor jRecordAccessor = raccessorToJObject.get(accessor);
+ if (jRecordAccessor == null) {
+ jRecordAccessor = new JRecordAccessor(accessor.getInputRecordType(), arg.getObjectPool());
+ raccessorToJObject.put(accessor, jRecordAccessor);
+ }
+ try {
+ result = jRecordAccessor.access(accessor, arg.getObjectPool(), (ARecordType) arg.getAtype(), this);
+ } catch (Exception e) {
+ throw new AsterixException(e);
+ }
+ return result;
+ }
+
+ @Override
+ public IJObject visit(AFlatValuePointable accessor, TypeInfo arg) throws AsterixException {
+ ATypeTag typeTag = arg.getTypeTag();
+ IJObject result = null;
+ IJObjectAccessor jObjectAccessor = flatJObjectAccessors.get(typeTag);
+ if (jObjectAccessor == null) {
+ jObjectAccessor = JObjectAccessors.createFlatJObjectAccessor(typeTag);
+ flatJObjectAccessors.put(typeTag, jObjectAccessor);
+ }
+
+ try {
+ result = jObjectAccessor.access(accessor, arg.getObjectPool());
+ } catch (HyracksDataException e) {
+ throw new AsterixException(e);
+ }
+ return result;
+ }
+
+}
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
index ff662ae..f5f404a 100644
--- 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
@@ -42,6 +42,7 @@
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.base.APoint;
import edu.uci.ics.asterix.om.types.AOrderedListType;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -52,6 +53,7 @@
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;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public class JObjectUtil {
@@ -129,7 +131,11 @@
long start = dis.readLong();
long end = dis.readLong();
byte intervalType = dis.readByte();
- ((JInterval) jObject).setValue(start, end, intervalType);
+ try {
+ ((JInterval) jObject).setValue(start, end, intervalType);
+ } catch (AlgebricksException e) {
+ throw new AsterixException(e);
+ }
break;
}
@@ -184,7 +190,7 @@
p1.setValue(dis.readDouble(), dis.readDouble());
points.add(p1);
}
- ((JPolygon) jObject).setValue(points);
+ ((JPolygon) jObject).setValue(points.toArray(new APoint[]{}));
break;
}
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
index 191fda6..61a60b6 100644
--- 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
@@ -16,21 +16,56 @@
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
+import java.io.DataOutput;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import edu.uci.ics.asterix.builders.IAsterixListBuilder;
+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.ABooleanSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ACircleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ALineSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.APoint3DSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.base.ADouble;
+import edu.uci.ics.asterix.om.base.AFloat;
+import edu.uci.ics.asterix.om.base.AInt16;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AInt64;
+import edu.uci.ics.asterix.om.base.AInt8;
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.AMutableInt16;
import edu.uci.ics.asterix.om.base.AMutableInt32;
import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.AMutableInt8;
import edu.uci.ics.asterix.om.base.AMutableInterval;
import edu.uci.ics.asterix.om.base.AMutableLine;
import edu.uci.ics.asterix.om.base.AMutableOrderedList;
@@ -43,15 +78,17 @@
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.ARectangle;
+import edu.uci.ics.asterix.om.base.AString;
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;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
public class JObjects {
@@ -73,26 +110,84 @@
public IAObject getIAObject() {
return value;
}
+
}
- public static final class JInt implements IJObject {
+ public static final class JByte extends JObject {
- private AMutableInt32 value;
+ public JByte(byte value) {
+ super(new AMutableInt8(value));
+ }
+
+ public void setValue(byte v) {
+ ((AMutableInt8) value).setValue(v);
+ }
+
+ public byte getValue() {
+ return ((AMutableInt8) value).getByteValue();
+ }
+
+ @Override
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(value.getType().getTypeTag().serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ AInt8SerializerDeserializer.INSTANCE.serialize((AInt8) value, dataOutput);
+ }
+
+ @Override
+ public void reset() {
+ ((AMutableInt8) value).setValue((byte) 0);
+ }
+ }
+
+ public static final class JShort extends JObject {
+
+ private AMutableInt16 value;
+
+ public JShort(short value) {
+ super(new AMutableInt16(value));
+ }
+
+ public void setValue(byte v) {
+ ((AMutableInt16) value).setValue(v);
+ }
+
+ public short getValue() {
+ return ((AMutableInt16) value).getShortValue();
+ }
+
+ @Override
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(value.getType().getTypeTag().serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ AInt16SerializerDeserializer.INSTANCE.serialize((AInt16) value, dataOutput);
+ }
+
+ @Override
+ public void reset() {
+ ((AMutableInt16) value).setValue((short) 0);
+ }
+
+ }
+
+ public static final class JInt extends JObject {
public JInt(int value) {
- this.value = new AMutableInt32(value);
+ super(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;
+ ((AMutableInt32) value).setValue(v);
}
public int getValue() {
@@ -100,15 +195,21 @@
}
@Override
- public ATypeTag getTypeTag() {
- return BuiltinType.AINT32.getTypeTag();
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(value.getType().getTypeTag().serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ AInt32SerializerDeserializer.INSTANCE.serialize((AInt32) value, dataOutput);
}
@Override
- public IAObject getIAObject() {
- return value;
+ public void reset() {
+ ((AMutableInt32) value).setValue(0);
}
-
}
public static final class JBoolean implements IJObject {
@@ -133,6 +234,23 @@
return value ? ABoolean.TRUE : ABoolean.FALSE;
}
+ @Override
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(ATypeTag.BOOLEAN.serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ ABooleanSerializerDeserializer.INSTANCE.serialize((ABoolean) getIAObject(), dataOutput);
+ }
+
+ @Override
+ public void reset() {
+ value = false;
+ }
+
}
public static final class JLong extends JObject {
@@ -149,6 +267,23 @@
return ((AMutableInt64) value).getLongValue();
}
+ @Override
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(value.getType().getTypeTag().serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ AInt64SerializerDeserializer.INSTANCE.serialize((AInt64) value, dataOutput);
+ }
+
+ @Override
+ public void reset() {
+ ((AMutableInt64) value).setValue(0);
+ }
+
}
public static final class JDouble extends JObject {
@@ -165,6 +300,23 @@
return ((AMutableDouble) value).getDoubleValue();
}
+ @Override
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(value.getType().getTypeTag().serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ ADoubleSerializerDeserializer.INSTANCE.serialize((ADouble) value, dataOutput);
+ }
+
+ @Override
+ public void reset() {
+ ((AMutableDouble) value).setValue(0);
+ }
+
}
public static final class JString extends JObject {
@@ -181,14 +333,29 @@
return ((AMutableString) value).getStringValue();
}
+ @Override
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(value.getType().getTypeTag().serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ AStringSerializerDeserializer.INSTANCE.serialize((AString) value, dataOutput);
+ }
+
+ @Override
+ public void reset() {
+ ((AMutableString) value).setValue("");
+ }
+
}
- public static final class JFloat implements IJObject {
-
- private AMutableFloat value;
+ public static final class JFloat extends JObject {
public JFloat(float v) {
- value = new AMutableFloat(v);
+ super(new AMutableFloat(v));
}
public void setValue(float v) {
@@ -200,13 +367,20 @@
}
@Override
- public ATypeTag getTypeTag() {
- return BuiltinType.AFLOAT.getTypeTag();
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(value.getType().getTypeTag().serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ AFloatSerializerDeserializer.INSTANCE.serialize((AFloat) value, dataOutput);
}
@Override
- public IAObject getIAObject() {
- return value;
+ public void reset() {
+ ((AMutableFloat) value).setValue(0);
}
}
@@ -237,18 +411,37 @@
public String toString() {
return value.toString();
}
+
+ @Override
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(value.getType().getTypeTag().serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ APointSerializerDeserializer.INSTANCE.serialize((APoint) value, dataOutput);
+ }
+
+ @Override
+ public void reset() {
+ ((AMutablePoint) value).setValue(0, 0);
+ }
}
- public static final class JRectangle implements IJObject {
-
- private AMutableRectangle rect;
+ public static final class JRectangle extends JObject {
public JRectangle(JPoint p1, JPoint p2) {
- rect = new AMutableRectangle((APoint) p1.getValue(), (APoint) p2.getValue());
+ super(new AMutableRectangle((APoint) p1.getIAObject(), (APoint) p2.getIAObject()));
}
public void setValue(JPoint p1, JPoint p2) {
- this.rect.setValue((APoint) p1.getValue(), (APoint) p2.getValue());
+ ((AMutableRectangle) value).setValue((APoint) p1.getValue(), (APoint) p2.getValue());
+ }
+
+ public void setValue(APoint p1, APoint p2) {
+ ((AMutableRectangle) value).setValue(p1, p2);
}
@Override
@@ -257,190 +450,200 @@
}
@Override
- public IAObject getIAObject() {
- return rect;
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(value.getType().getTypeTag().serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ ARectangleSerializerDeserializer.INSTANCE.serialize((ARectangle) value, dataOutput);
}
@Override
- public String toString() {
- return rect.toString();
+ public void reset() {
}
}
- public static final class JTime implements IJObject {
-
- private AMutableTime time;
+ public static final class JTime extends JObject {
public JTime(int timeInMillsec) {
- time = new AMutableTime(timeInMillsec);
+ super(new AMutableTime(timeInMillsec));
}
public void setValue(int timeInMillsec) {
- time.setValue(timeInMillsec);
+ ((AMutableTime) value).setValue(timeInMillsec);
}
@Override
- public ATypeTag getTypeTag() {
- return ATypeTag.TIME;
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(ATypeTag.TIME.serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ ATimeSerializerDeserializer.INSTANCE.serialize((AMutableTime) value, dataOutput);
}
@Override
- public IAObject getIAObject() {
- return time;
- }
-
- @Override
- public String toString() {
- return time.toString();
+ public void reset() {
+ ((AMutableTime) value).setValue(0);
}
}
- public static final class JInterval implements IJObject {
-
- private AMutableInterval interval;
+ public static final class JInterval extends JObject {
public JInterval(long intervalStart, long intervalEnd) {
- interval = new AMutableInterval(intervalStart, intervalEnd, (byte) 0);
+ super(new AMutableInterval(intervalStart, intervalEnd, (byte) 0));
}
- public void setValue(long intervalStart, long intervalEnd, byte typetag) throws AsterixException {
- try {
- interval.setValue(intervalStart, intervalEnd, typetag);
- } catch (AlgebricksException e) {
- throw new AsterixException(e);
- }
- }
-
- @Override
- public ATypeTag getTypeTag() {
- return ATypeTag.INTERVAL;
- }
-
- @Override
- public IAObject getIAObject() {
- return interval;
- }
-
- @Override
- public String toString() {
- return interval.toString();
+ public void setValue(long intervalStart, long intervalEnd, byte typetag) throws AlgebricksException {
+ ((AMutableInterval) value).setValue(intervalStart, intervalEnd, typetag);
}
public long getIntervalStart() {
- return interval.getIntervalStart();
+ return ((AMutableInterval) value).getIntervalStart();
}
public long getIntervalEnd() {
- return interval.getIntervalEnd();
+ return ((AMutableInterval) value).getIntervalEnd();
}
public short getIntervalType() {
- return interval.getIntervalType();
+ return ((AMutableInterval) value).getIntervalType();
+ }
+
+ @Override
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(ATypeTag.INTERVAL.serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ AIntervalSerializerDeserializer.INSTANCE.serialize(((AMutableInterval) value), dataOutput);
+ }
+
+ @Override
+ public void reset() throws AlgebricksException {
+ ((AMutableInterval) value).setValue(0L, 0L, (byte) 0);
}
}
- public static final class JDate implements IJObject {
-
- private AMutableDate date;
+ public static final class JDate extends JObject {
public JDate(int chrononTimeInDays) {
- date = new AMutableDate(chrononTimeInDays);
+ super(new AMutableDate(chrononTimeInDays));
}
public void setValue(int chrononTimeInDays) {
- date.setValue(chrononTimeInDays);
+ ((AMutableDate) value).setValue(chrononTimeInDays);
}
@Override
- public ATypeTag getTypeTag() {
- return ATypeTag.DATE;
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(ATypeTag.DATE.serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ ADateSerializerDeserializer.INSTANCE.serialize(((AMutableDate) value), dataOutput);
}
@Override
- public IAObject getIAObject() {
- return date;
+ public void reset() {
+ ((AMutableDate) value).setValue(0);
}
-
- @Override
- public String toString() {
- return date.toString();
- }
-
}
- public static final class JDateTime implements IJObject {
-
- private AMutableDateTime dateTime;
+ public static final class JDateTime extends JObject {
public JDateTime(long chrononTime) {
- dateTime = new AMutableDateTime(chrononTime);
+ super(new AMutableDateTime(chrononTime));
}
public void setValue(long chrononTime) {
- dateTime.setValue(chrononTime);
+ ((AMutableDateTime) value).setValue(chrononTime);
}
@Override
- public ATypeTag getTypeTag() {
- return ATypeTag.DATETIME;
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(ATypeTag.DATETIME.serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ ADateTimeSerializerDeserializer.INSTANCE.serialize(((AMutableDateTime) value), dataOutput);
}
@Override
- public IAObject getIAObject() {
- return dateTime;
- }
-
- @Override
- public String toString() {
- return dateTime.toString();
+ public void reset() {
+ ((AMutableDateTime) value).setValue(0);
}
}
- public static final class JDuration implements IJObject {
-
- private AMutableDuration duration;
+ public static final class JDuration extends JObject {
public JDuration(int months, long milliseconds) {
- duration = new AMutableDuration(months, milliseconds);
+ super(new AMutableDuration(months, milliseconds));
}
public void setValue(int months, long milliseconds) {
- duration.setValue(months, milliseconds);
+ ((AMutableDuration) value).setValue(months, milliseconds);
}
@Override
- public ATypeTag getTypeTag() {
- return ATypeTag.DURATION;
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(ATypeTag.DURATION.serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ ADurationSerializerDeserializer.INSTANCE.serialize(((AMutableDuration) value), dataOutput);
}
@Override
- public IAObject getIAObject() {
- return duration;
- }
-
- @Override
- public String toString() {
- return duration.toString();
+ public void reset() {
+ ((AMutableDuration) value).setValue(0, 0);
}
}
- public static final class JPolygon implements IJObject {
+ public static final class JPolygon extends JObject {
- private AMutablePolygon polygon;
- private List<JPoint> points;
-
- public JPolygon(List<JPoint> points) {
- this.points = points;
+ public JPolygon(JPoint[] points) {
+ super(new AMutablePolygon(getAPoints(points)));
}
- public void setValue(List<JPoint> points) {
- this.points = points;
- polygon = null;
+ public void setValue(APoint[] points) {
+ ((AMutablePolygon) value).setValue(points);
+ }
+
+ public void setValue(JPoint[] points) {
+ ((AMutablePolygon) value).setValue(getAPoints(points));
+ }
+
+ private static APoint[] getAPoints(JPoint[] jpoints) {
+ APoint[] apoints = new APoint[jpoints.length];
+ int index = 0;
+ for (JPoint jpoint : jpoints) {
+ apoints[index++] = (APoint) jpoint.getIAObject();
+ }
+ return apoints;
}
@Override
@@ -449,35 +652,32 @@
}
@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();
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(ATypeTag.POLYGON.serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
}
- polygon = new AMutablePolygon(pts);
}
- return polygon;
+ APolygonSerializerDeserializer.INSTANCE.serialize((AMutablePolygon) value, dataOutput);
}
@Override
- public String toString() {
- return getIAObject().toString();
+ public void reset() {
+ ((AMutablePolygon) value).setValue(null);
}
}
- public static final class JCircle implements IJObject {
-
- private AMutableCircle circle;
+ public static final class JCircle extends JObject {
public JCircle(JPoint center, double radius) {
- circle = new AMutableCircle((APoint) center.getIAObject(), radius);
+ super(new AMutableCircle((APoint) center.getIAObject(), radius));
}
public void setValue(JPoint center, double radius) {
- circle.setValue((APoint) center.getIAObject(), radius);
+ ((AMutableCircle) (value)).setValue((APoint) center.getIAObject(), radius);
}
@Override
@@ -486,56 +686,64 @@
}
@Override
- public IAObject getIAObject() {
- return circle;
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(ATypeTag.CIRCLE.serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ ACircleSerializerDeserializer.INSTANCE.serialize(((AMutableCircle) (value)), dataOutput);
}
@Override
- public String toString() {
- return circle.toString();
+ public void reset() {
}
-
}
- public static final class JLine implements IJObject {
-
- private AMutableLine line;
+ public static final class JLine extends JObject {
public JLine(JPoint p1, JPoint p2) {
- line = new AMutableLine((APoint) p1.getIAObject(), (APoint) p2.getIAObject());
+ super(new AMutableLine((APoint) p1.getIAObject(), (APoint) p2.getIAObject()));
}
public void setValue(JPoint p1, JPoint p2) {
- line.setValue((APoint) p1.getIAObject(), (APoint) p2.getIAObject());
+ ((AMutableLine) value).setValue((APoint) p1.getIAObject(), (APoint) p2.getIAObject());
+ }
+
+ public void setValue(APoint p1, APoint p2) {
+ ((AMutableLine) value).setValue(p1, p2);
}
@Override
- public ATypeTag getTypeTag() {
- return ATypeTag.LINE;
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(ATypeTag.LINE.serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ ALineSerializerDeserializer.INSTANCE.serialize(((AMutableLine) value), dataOutput);
}
@Override
- public IAObject getIAObject() {
- return line;
- }
+ public void reset() {
+ // TODO Auto-generated method stub
- @Override
- public String toString() {
- return line.toString();
}
}
- public static final class JPoint3D implements IJObject {
-
- private AMutablePoint3D value;
+ public static final class JPoint3D extends JObject {
public JPoint3D(double x, double y, double z) {
- value = new AMutablePoint3D(x, y, z);
+ super(new AMutablePoint3D(x, y, z));
}
public void setValue(double x, double y, double z) {
- value.setValue(x, y, z);
+ ((AMutablePoint3D) value).setValue(x, y, z);
}
public double getXValue() {
@@ -550,40 +758,75 @@
return ((AMutablePoint3D) value).getZ();
}
- public IAObject getValue() {
- return value;
+ @Override
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ if (writeTypeTag) {
+ try {
+ dataOutput.writeByte(ATypeTag.POINT3D.serialize());
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ APoint3DSerializerDeserializer.INSTANCE.serialize(((AMutablePoint3D) value), dataOutput);
}
@Override
- public String toString() {
- return value.toString();
- }
+ public void reset() {
+ // TODO Auto-generated method stub
- @Override
- public ATypeTag getTypeTag() {
- return ATypeTag.POINT3D;
- }
-
- @Override
- public IAObject getIAObject() {
- return value;
}
}
- public static final class JOrderedList implements IJObject {
+ public static abstract class JList implements IJObject {
+ protected List<IJObject> jObjects;
- private AOrderedListType listType;
- private List<IJObject> jObjects;
+ public JList() {
+ jObjects = new ArrayList<IJObject>();
+ }
- public JOrderedList(IJObject jObject) {
- this.listType = new AOrderedListType(jObject.getIAObject().getType(), null);
- this.jObjects = new ArrayList<IJObject>();
+ public boolean isEmpty() {
+ return jObjects.isEmpty();
}
public void add(IJObject jObject) {
jObjects.add(jObject);
}
+ 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 Iterator<IJObject> iterator() {
+ return jObjects.iterator();
+ }
+ }
+
+ public static final class JOrderedList extends JList {
+
+ private AOrderedListType listType;
+
+ public JOrderedList(IJObject jObject) {
+ super();
+ this.listType = new AOrderedListType(jObject.getIAObject().getType(), null);
+ }
+
+ public JOrderedList(IAType listItemType) {
+ super();
+ this.listType = new AOrderedListType(listItemType, null);
+ }
+
@Override
public ATypeTag getTypeTag() {
return ATypeTag.ORDEREDLIST;
@@ -602,34 +845,43 @@
return listType;
}
- public void addAll(Collection<IJObject> jObjectCollection) {
- jObjects.addAll(jObjectCollection);
+ @Override
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ IAsterixListBuilder listBuilder = new UnorderedListBuilder();
+ listBuilder.reset(listType);
+ ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+ for (IJObject jObject : jObjects) {
+ fieldValue.reset();
+ jObject.serialize(fieldValue.getDataOutput(), true);
+ listBuilder.addItem(fieldValue);
+ }
+ listBuilder.write(dataOutput, writeTypeTag);
+
}
- public void clear() {
- jObjects.clear();
- }
+ @Override
+ public void reset() {
+ // TODO Auto-generated method stub
- public IJObject getElement(int index) {
- return jObjects.get(index);
- }
-
- public int size() {
- return jObjects.size();
}
}
- public static final class JUnorderedList implements IJObject {
+ public static final class JUnorderedList extends JList {
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 JUnorderedList(IAType listItemType) {
+ super();
+ this.listType = new AUnorderedListType(listItemType, null);
+ this.jObjects = new ArrayList<IJObject>();
+ }
+
public void add(IJObject jObject) {
jObjects.add(jObject);
}
@@ -652,60 +904,43 @@
return listType;
}
- public boolean isEmpty() {
- return jObjects.isEmpty();
+ @Override
+ public void serialize(DataOutput dataOutput, boolean writeTypeTag) throws HyracksDataException {
+ IAsterixListBuilder listBuilder = new UnorderedListBuilder();
+ listBuilder.reset(listType);
+ ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+ for (IJObject jObject : jObjects) {
+ fieldValue.reset();
+ jObject.serialize(fieldValue.getDataOutput(), true);
+ listBuilder.addItem(fieldValue);
+ }
+ listBuilder.write(dataOutput, writeTypeTag);
}
- public void addAll(Collection<IJObject> jObjectCollection) {
- jObjects.addAll(jObjectCollection);
- }
-
- public void clear() {
+ @Override
+ public void reset() {
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();
- }
+ private IJObject[] fields;
+ private Map<String, IJObject> openFields;
public JRecord(ARecordType recordType, IJObject[] fields) {
this.recordType = recordType;
- this.fields = new ArrayList<IJObject>();
- for (IJObject jObject : fields) {
- this.fields.add(jObject);
- }
- initFieldInfo();
+ this.fields = fields;
+ this.openFields = new LinkedHashMap<String, IJObject>();
}
- 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();
+ public JRecord(ARecordType recordType, IJObject[] fields, LinkedHashMap<String, IJObject> openFields) {
+ this.recordType = recordType;
+ this.fields = fields;
+ this.openFields = openFields;
}
private ARecordType getARecordType(String[] fieldNames, IJObject[] fields) throws AsterixException {
@@ -723,74 +958,34 @@
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) {
+ public void addField(String fieldName, IJObject fieldValue) throws AsterixException {
int pos = getFieldPosByName(fieldName);
if (pos >= 0) {
- throw new IllegalArgumentException("field already defined");
+ throw new AsterixException("field already defined in closed part");
}
- numFieldsAdded++;
- fields.add(fieldValue);
- fieldNames.add(fieldName);
- fieldTypes.add(fieldValue.getIAObject().getType());
- openField.add(true);
+ if (openFields.get(fieldName) != null) {
+ throw new AsterixException("field already defined in open part");
+ }
+ openFields.put(fieldName, fieldValue);
}
public IJObject getValueByName(String fieldName) throws AsterixException, IOException {
+ // check closed part
int fieldPos = getFieldPosByName(fieldName);
- if (fieldPos < 0) {
- throw new AsterixException("unknown field: " + fieldName);
+ if (fieldPos >= 0) {
+ return fields[fieldPos];
+ } else {
+ // check open part
+ IJObject fieldValue = openFields.get(fieldName);
+ if (fieldValue == null) {
+ throw new AsterixException("unknown field: " + fieldName);
+ }
+ return fieldValue;
}
- 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();
+ public void setValueAtPos(int pos, IJObject jObject) {
+ fields[pos] = jObject;
}
@Override
@@ -798,16 +993,22 @@
return recordType.getTypeTag();
}
- public void setField(String fieldName, IJObject fieldValue) {
+ public void setField(String fieldName, IJObject fieldValue) throws AsterixException {
int pos = getFieldPosByName(fieldName);
- fields.set(pos, fieldValue);
- if (value != null) {
- value.setValueAtPos(pos, fieldValue.getIAObject());
+ if (pos >= 0) {
+ fields[pos] = fieldValue;
+ } else {
+ if (openFields.get(fieldName) != null) {
+ openFields.put(fieldName, fieldValue);
+ } else {
+ throw new AsterixException("unknown field: " + fieldName);
+ }
}
}
private int getFieldPosByName(String fieldName) {
int index = 0;
+ String[] fieldNames = recordType.getFieldNames();
for (String name : fieldNames) {
if (name.equals(fieldName)) {
return index;
@@ -821,40 +1022,75 @@
return recordType;
}
- public List<IJObject> getFields() {
+ public IJObject[] getFields() {
return fields;
}
+ public RecordBuilder getRecordBuilder() {
+ RecordBuilder recordBuilder = new RecordBuilder();
+ recordBuilder.reset(recordType);
+ return recordBuilder;
+ }
+
+ public void serialize(DataOutput output, boolean writeTypeTag) throws HyracksDataException {
+ RecordBuilder recordBuilder = new RecordBuilder();
+ recordBuilder.reset(recordType);
+ int index = 0;
+ ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
+ for (IJObject jObject : fields) {
+ fieldValue.reset();
+ jObject.serialize(fieldValue.getDataOutput(), writeTypeTag);
+ recordBuilder.addField(index, fieldValue);
+ index++;
+ }
+
+ try {
+ if (openFields != null && !openFields.isEmpty()) {
+ ArrayBackedValueStorage openFieldName = new ArrayBackedValueStorage();
+ ArrayBackedValueStorage openFieldValue = new ArrayBackedValueStorage();
+ AMutableString nameValue = new AMutableString(""); // get from the pool
+ for (Entry<String, IJObject> entry : openFields.entrySet()) {
+ openFieldName.reset();
+ openFieldValue.reset();
+ nameValue.setValue(entry.getKey());
+ openFieldName.getDataOutput().write(ATypeTag.STRING.serialize());
+ AStringSerializerDeserializer.INSTANCE.serialize(nameValue, openFieldName.getDataOutput());
+ entry.getValue().serialize(openFieldValue.getDataOutput(), true);
+ recordBuilder.addField(openFieldName, openFieldValue);
+ }
+ }
+ } catch (IOException | AsterixException ae) {
+ throw new HyracksDataException(ae);
+ }
+ try {
+ recordBuilder.write(output, writeTypeTag);
+ } catch (IOException | AsterixException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
@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);
+ public void reset() throws AlgebricksException {
+ if (openFields != null && !openFields.isEmpty()) {
+ openFields.clear();
+ }
+ if (fields != null) {
+ for (IJObject field : fields) {
+ if (field != null) {
+ field.reset();
+ }
}
- numFieldsAdded = 0;
}
}
- public List<Boolean> getOpenField() {
- return openField;
- }
-
- public List<String> getFieldNames() {
- return fieldNames;
- }
-
- public List<IAType> getFieldTypes() {
- return fieldTypes;
+ public void reset(IJObject[] fields, LinkedHashMap<String, IJObject> openFields) throws AlgebricksException {
+ this.reset();
+ this.fields = fields;
+ this.openFields = openFields;
}
}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/Datatypes.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/Datatypes.java
new file mode 100644
index 0000000..e126bf6
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/Datatypes.java
@@ -0,0 +1,21 @@
+package edu.uci.ics.asterix.external.util;
+
+public class Datatypes {
+
+ public static final class Tweet {
+ public static final String ID = "id";
+ public static final String USER = "user";
+ public static final String MESSAGE = "message_text";
+ public static final String LATITUDE = "latitude";
+ public static final String LONGITUDE = "longitude";
+ public static final String CREATED_AT = "created_at";
+ public static final String SCREEN_NAME = "screen_name";
+ public static final String COUNTRY = "country";
+ }
+
+ public static final class ProcessedTweet {
+ public static final String USER_NAME = "user_name";
+ public static final String LOCATION = "location";
+ public static final String TOPICS = "topics";
+ }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/TweetProcessor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/TweetProcessor.java
new file mode 100644
index 0000000..092d715
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/TweetProcessor.java
@@ -0,0 +1,72 @@
+package edu.uci.ics.asterix.external.util;
+
+import twitter4j.Status;
+import twitter4j.User;
+import edu.uci.ics.asterix.om.base.AMutableDouble;
+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.types.ARecordType;
+
+public class TweetProcessor {
+
+ private IAObject[] mutableTweetFields;
+ private IAObject[] mutableUserFields;
+ private AMutableRecord mutableRecord;
+ private AMutableRecord mutableUser;
+
+ public TweetProcessor(ARecordType recordType) {
+ mutableUserFields = new IAObject[] { new AMutableString(null), new AMutableString(null), new AMutableInt32(0),
+ new AMutableInt32(0), new AMutableString(null), new AMutableInt32(0) };
+ mutableUser = new AMutableRecord((ARecordType) recordType.getFieldTypes()[1], mutableUserFields);
+
+ mutableTweetFields = new IAObject[] { new AMutableString(null), mutableUser, new AMutableDouble(0),
+ new AMutableDouble(0), new AMutableString(null), new AMutableString(null) };
+ mutableRecord = new AMutableRecord(recordType, mutableTweetFields);
+
+ }
+
+ public AMutableRecord processNextTweet(Status tweet) {
+ User user = tweet.getUser();
+ ((AMutableString) mutableUserFields[0]).setValue(getNormalizedString(user.getScreenName()));
+ ((AMutableString) mutableUserFields[1]).setValue(getNormalizedString(user.getLang()));
+ ((AMutableInt32) mutableUserFields[2]).setValue(user.getFriendsCount());
+ ((AMutableInt32) mutableUserFields[3]).setValue(user.getStatusesCount());
+ ((AMutableString) mutableUserFields[4]).setValue(getNormalizedString(user.getName()));
+ ((AMutableInt32) mutableUserFields[5]).setValue(user.getFollowersCount());
+
+ ((AMutableString) mutableTweetFields[0]).setValue(tweet.getId() + "");
+
+ for (int i = 0; i < 6; i++) {
+ ((AMutableRecord) mutableTweetFields[1]).setValueAtPos(i, mutableUserFields[i]);
+ }
+ if (tweet.getGeoLocation() != null) {
+ ((AMutableDouble) mutableTweetFields[2]).setValue(tweet.getGeoLocation().getLatitude());
+ ((AMutableDouble) mutableTweetFields[3]).setValue(tweet.getGeoLocation().getLongitude());
+ } else {
+ ((AMutableDouble) mutableTweetFields[2]).setValue(0);
+ ((AMutableDouble) mutableTweetFields[3]).setValue(0);
+ }
+ ((AMutableString) mutableTweetFields[4]).setValue(getNormalizedString(tweet.getCreatedAt().toString()));
+ ((AMutableString) mutableTweetFields[5]).setValue(getNormalizedString(tweet.getText()));
+
+ for (int i = 0; i < 6; i++) {
+ mutableRecord.setValueAtPos(i, mutableTweetFields[i]);
+ }
+
+ return mutableRecord;
+
+ }
+
+ public static String getNormalizedString(String originalString) {
+ String asciiText = originalString.replaceAll("[^\\x00-\\x7F]", "").replaceAll("\n", " ");
+ return asciiText.trim();
+
+ }
+
+ public AMutableRecord getMutableRecord() {
+ return mutableRecord;
+ }
+
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/TwitterUtil.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/TwitterUtil.java
new file mode 100644
index 0000000..bd1d75c
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/TwitterUtil.java
@@ -0,0 +1,143 @@
+package edu.uci.ics.asterix.external.util;
+
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import twitter4j.FilterQuery;
+import twitter4j.Twitter;
+import twitter4j.TwitterFactory;
+import twitter4j.TwitterStream;
+import twitter4j.TwitterStreamFactory;
+import twitter4j.conf.ConfigurationBuilder;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+
+public class TwitterUtil {
+
+ public static class ConfigurationConstants {
+ public static final String KEY_LOCATION = "location";
+ public static final String LOCATION_US = "US";
+ }
+
+ public static class GeoConstants {
+ public static Map<String, double[][]> boundingBoxes = initializeBoundingBoxes();
+ public static final double[][] US = new double[][] { { -124.848974, 24.396308 }, { -66.885444, 49.384358 } };
+ }
+
+ private static Map<String, double[][]> initializeBoundingBoxes() {
+ Map<String, double[][]> boundingBoxes = new HashMap<String, double[][]>();
+ boundingBoxes.put(ConfigurationConstants.LOCATION_US, new double[][] { { -124.848974, 24.396308 },
+ { -66.885444, 49.384358 } });
+ return boundingBoxes;
+ }
+
+ public static FilterQuery getFilterQuery(Map<String, String> configuration) throws AsterixException {
+ String locationValue = configuration.get(ConfigurationConstants.KEY_LOCATION);
+ double[][] locations = null;
+ if (locationValue != null) {
+ if (locationValue.contains(",")) {
+ String[] coordinatesString = locationValue.trim().split(",");
+ locations = new double[2][2];
+ for (int i = 0; i < 2; i++) {
+ for (int j = 0; j < 2; j++) {
+ try {
+ locations[i][j] = Double.parseDouble(coordinatesString[2 * i + j]);
+ } catch (NumberFormatException ne) {
+ throw new AsterixException("Incorrect coordinate value " + coordinatesString[2 * i + j]);
+ }
+ }
+ }
+ } else {
+ locations = GeoConstants.boundingBoxes.get(locationValue);
+ }
+ if (locations != null) {
+ FilterQuery filterQuery = new FilterQuery();
+ filterQuery.locations(locations);
+ return filterQuery;
+ }
+ }
+ return null;
+
+ }
+
+ public static Twitter getTwitterService(Map<String, String> configuration) {
+ ConfigurationBuilder cb = getAuthConfiguration(configuration);
+ TwitterFactory tf = new TwitterFactory(cb.build());
+ Twitter twitter = tf.getInstance();
+ return twitter;
+ }
+
+ public static TwitterStream getTwitterStream(Map<String, String> configuration) {
+ ConfigurationBuilder cb = getAuthConfiguration(configuration);
+ TwitterStreamFactory factory = new TwitterStreamFactory(cb.build());
+ return factory.getInstance();
+ }
+
+ private static ConfigurationBuilder getAuthConfiguration(Map<String, String> configuration) {
+ ConfigurationBuilder cb = new ConfigurationBuilder();
+ cb.setDebugEnabled(true);
+ String oAuthConsumerKey = configuration.get(AuthenticationConstants.OAUTH_CONSUMER_KEY);
+ String oAuthConsumerSecret = configuration.get(AuthenticationConstants.OAUTH_CONSUMER_SECRET);
+ String oAuthAccessToken = configuration.get(AuthenticationConstants.OAUTH_ACCESS_TOKEN);
+ String oAuthAccessTokenSecret = configuration.get(AuthenticationConstants.OAUTH_ACCESS_TOKEN_SECRET);
+
+ cb.setOAuthConsumerKey(oAuthConsumerKey);
+ cb.setOAuthConsumerSecret(oAuthConsumerSecret);
+ cb.setOAuthAccessToken(oAuthAccessToken);
+ cb.setOAuthAccessTokenSecret(oAuthAccessTokenSecret);
+ return cb;
+ }
+
+ public static void initializeConfigurationWithAuthInfo(Map<String, String> configuration) throws AsterixException {
+ String authMode = configuration.get(AuthenticationConstants.AUTHENTICATION_MODE);
+ if (authMode == null) {
+ authMode = AuthenticationConstants.AUTHENTICATION_MODE_FILE;
+ }
+ try {
+ switch (authMode) {
+ case AuthenticationConstants.AUTHENTICATION_MODE_FILE:
+ Properties prop = new Properties();
+ String authFile = configuration.get(AuthenticationConstants.OAUTH_AUTHENTICATION_FILE);
+ if (authFile == null) {
+ authFile = AuthenticationConstants.DEFAULT_AUTH_FILE;
+ }
+ InputStream in = TwitterUtil.class.getResourceAsStream(authFile);
+ prop.load(in);
+ in.close();
+ configuration.put(AuthenticationConstants.OAUTH_CONSUMER_KEY,
+ prop.getProperty(AuthenticationConstants.OAUTH_CONSUMER_KEY));
+ configuration.put(AuthenticationConstants.OAUTH_CONSUMER_SECRET,
+ prop.getProperty(AuthenticationConstants.OAUTH_CONSUMER_SECRET));
+ configuration.put(AuthenticationConstants.OAUTH_ACCESS_TOKEN,
+ prop.getProperty(AuthenticationConstants.OAUTH_ACCESS_TOKEN));
+ configuration.put(AuthenticationConstants.OAUTH_ACCESS_TOKEN_SECRET,
+ prop.getProperty(AuthenticationConstants.OAUTH_ACCESS_TOKEN_SECRET));
+ break;
+ case AuthenticationConstants.AUTHENTICATION_MODE_EXPLICIT:
+ break;
+ }
+ } catch (Exception e) {
+ throw new AsterixException("Incorrect configuration! unable to load authentication credentials "
+ + e.getMessage());
+ }
+ }
+
+ public static final class AuthenticationConstants {
+ public static final String OAUTH_CONSUMER_KEY = "consumer.key";
+ public static final String OAUTH_CONSUMER_SECRET = "consumer.secret";
+ public static final String OAUTH_ACCESS_TOKEN = "access.token";
+ public static final String OAUTH_ACCESS_TOKEN_SECRET = "access.token.secret";
+ public static final String OAUTH_AUTHENTICATION_FILE = "authentication.file";
+ public static final String AUTHENTICATION_MODE = "authentication.mode";
+ public static final String AUTHENTICATION_MODE_FILE = "file";
+ public static final String AUTHENTICATION_MODE_EXPLICIT = "explicit";
+ public static final String DEFAULT_AUTH_FILE = "/feed/twitter/auth.properties"; // default authentication file
+ }
+
+ public static final class SearchAPIConstants {
+ public static final String QUERY = "query";
+ public static final String INTERVAL = "interval";
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AddHashTagsFactory.java
similarity index 63%
copy from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AddHashTagsFactory.java
index 65ac354..90fddcd 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AddHashTagsFactory.java
@@ -3,27 +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.metadata.api;
+package edu.uci.ics.asterix.external.library;
-public interface IClusterManagementWork {
+import edu.uci.ics.asterix.external.library.IExternalScalarFunction;
+import edu.uci.ics.asterix.external.library.IFunctionFactory;
- public enum WorkType {
- ADD_NODE,
- REMOVE_NODE
+public class AddHashTagsFactory implements IFunctionFactory {
+
+ @Override
+ public IExternalScalarFunction getExternalFunction() {
+ return new AddHashTagsFunction();
}
- public WorkType getClusterManagementWorkType();
-
- public int getWorkId();
-
- public IClusterEventsSubscriber getSourceSubscriber();
}
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AddHashTagsFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AddHashTagsFunction.java
new file mode 100644
index 0000000..93a87f5
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AddHashTagsFunction.java
@@ -0,0 +1,76 @@
+/*
+ * 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.JDouble;
+import edu.uci.ics.asterix.external.library.java.JObjects.JPoint;
+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.JUnorderedList;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+import edu.uci.ics.asterix.external.util.Datatypes;
+
+public class AddHashTagsFunction implements IExternalScalarFunction {
+
+ private JUnorderedList list = null;
+ private JPoint location = null;
+
+ @Override
+ public void initialize(IFunctionHelper functionHelper) {
+ list = new JUnorderedList(functionHelper.getObject(JTypeTag.STRING));
+ location = new JPoint(0, 0);
+ }
+
+ @Override
+ public void deinitialize() {
+ }
+
+ @Override
+ public void evaluate(IFunctionHelper functionHelper) throws Exception {
+ list.clear();
+ JRecord inputRecord = (JRecord) functionHelper.getArgument(0);
+ JString text = (JString) inputRecord.getValueByName(Datatypes.Tweet.MESSAGE);
+ JDouble latitude = (JDouble) inputRecord.getValueByName(Datatypes.Tweet.LATITUDE);
+ JDouble longitude = (JDouble) inputRecord.getValueByName(Datatypes.Tweet.LONGITUDE);
+
+ if (latitude != null && longitude != null) {
+ location.setValue(latitude.getValue(), longitude.getValue());
+ }
+ 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 outputRecord = (JRecord) functionHelper.getResultObject();
+ outputRecord.setField(Datatypes.Tweet.ID, inputRecord.getValueByName(Datatypes.Tweet.ID));
+
+ JRecord userRecord = (JRecord) inputRecord.getValueByName(Datatypes.Tweet.USER);
+ outputRecord.setField(Datatypes.ProcessedTweet.USER_NAME,
+ userRecord.getValueByName(Datatypes.Tweet.SCREEN_NAME));
+
+ outputRecord.setField(Datatypes.ProcessedTweet.LOCATION, location);
+ outputRecord.setField(Datatypes.Tweet.CREATED_AT, inputRecord.getValueByName(Datatypes.Tweet.CREATED_AT));
+ outputRecord.setField(Datatypes.Tweet.MESSAGE, text);
+ outputRecord.setField(Datatypes.ProcessedTweet.TOPICS, list);
+
+ inputRecord.addField(Datatypes.ProcessedTweet.TOPICS, list);
+ functionHelper.setResult(outputRecord);
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AddHashTagsInPlaceFactory.java
similarity index 67%
copy from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
copy to asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AddHashTagsInPlaceFactory.java
index 65ac354..a2bec69 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AddHashTagsInPlaceFactory.java
@@ -3,27 +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.metadata.api;
+package edu.uci.ics.asterix.external.library;
-public interface IClusterManagementWork {
- public enum WorkType {
- ADD_NODE,
- REMOVE_NODE
+public class AddHashTagsInPlaceFactory implements IFunctionFactory {
+
+ @Override
+ public IExternalScalarFunction getExternalFunction() {
+ return new AddHashTagsInPlaceFunction();
}
- public WorkType getClusterManagementWorkType();
-
- public int getWorkId();
-
- public IClusterEventsSubscriber getSourceSubscriber();
}
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AddHashTagsInPlaceFunction.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AddHashTagsInPlaceFunction.java
new file mode 100644
index 0000000..a3f6f702
--- /dev/null
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/AddHashTagsInPlaceFunction.java
@@ -0,0 +1,54 @@
+/* 1
+ * Copyright 2009-2013 by The Regents of the University of California 2
+ * Licensed under the Apache License, Version 2.0 (the "License"); 3
+ * you may not use this file except in compliance with the License. 4
+ * you may obtain a copy of the License from 5
+ * 6
+ * 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.JRecord;
+import edu.uci.ics.asterix.external.library.java.JObjects.JString;
+import edu.uci.ics.asterix.external.library.java.JObjects.JUnorderedList;
+import edu.uci.ics.asterix.external.library.java.JTypeTag;
+import edu.uci.ics.asterix.external.util.Datatypes;
+
+public class AddHashTagsInPlaceFunction implements IExternalScalarFunction {
+
+ private JUnorderedList list = null;
+
+ @Override
+ public void initialize(IFunctionHelper functionHelper) {
+ list = new JUnorderedList(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 text = (JString) inputRecord.getValueByName(Datatypes.Tweet.MESSAGE);
+
+ 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);
+ }
+ }
+ inputRecord.addField(Datatypes.ProcessedTweet.TOPICS, list);
+ functionHelper.setResult(inputRecord);
+ }
+
+}
\ No newline at end of file
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
index 58995c2..ec04541 100644
--- 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
@@ -53,9 +53,6 @@
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/java/edu/uci/ics/asterix/external/library/adapter/TestTypedAdapter.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/adapter/TestTypedAdapter.java
index 07f1a40..7a2597e 100644
--- a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/adapter/TestTypedAdapter.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/adapter/TestTypedAdapter.java
@@ -23,8 +23,9 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import edu.uci.ics.asterix.common.feeds.api.IFeedAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IFeedAdapter.DataExchangeMode;
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;
@@ -43,8 +44,8 @@
private DummyGenerator generator;
public TestTypedAdapter(ITupleParserFactory parserFactory, ARecordType sourceDatatype, IHyracksTaskContext ctx,
- Map<String, String> configuration) throws IOException {
- super(parserFactory, sourceDatatype, ctx);
+ Map<String, String> configuration, int partition) throws IOException {
+ super(parserFactory, sourceDatatype, ctx, partition);
pos = new PipedOutputStream();
pis = new PipedInputStream(pos);
this.configuration = configuration;
@@ -131,4 +132,9 @@
generator.stop();
}
+ @Override
+ public boolean handleException(Exception e) {
+ return false;
+ }
+
}
diff --git a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/adapter/TestTypedAdapterFactory.java b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/adapter/TestTypedAdapterFactory.java
index b042e9c..5416ce2 100644
--- a/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/adapter/TestTypedAdapterFactory.java
+++ b/asterix-external-data/src/test/java/edu/uci/ics/asterix/external/library/adapter/TestTypedAdapterFactory.java
@@ -17,19 +17,23 @@
import java.util.Map;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
-import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter.AdapterType;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory.SupportedOperation;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapterFactory;
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.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
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;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-public class TestTypedAdapterFactory implements ITypedAdapterFactory {
+public class TestTypedAdapterFactory implements IFeedAdapterFactory {
/**
*
@@ -38,7 +42,7 @@
public static final String NAME = "test_typed_adapter";
- private static ARecordType adapterOutputType = initOutputType();
+ private ARecordType outputType;
public static final String KEY_NUM_OUTPUT_RECORDS = "num_output_records";
@@ -50,7 +54,7 @@
}
private static ARecordType initOutputType() {
- String[] fieldNames = new String[] { "tweetid", "message-text" };
+ String[] fieldNames = new String[] { "id", "message-text" };
IAType[] fieldTypes = new IAType[] { BuiltinType.AINT64, BuiltinType.ASTRING };
ARecordType outputType = null;
try {
@@ -67,29 +71,36 @@
}
@Override
- public AdapterType getAdapterType() {
- return AdapterType.TYPED;
- }
-
- @Override
public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
return new AlgebricksCountPartitionConstraint(1);
}
@Override
public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
- ITupleParserFactory tupleParserFactory = new AdmSchemafullRecordParserFactory(adapterOutputType);
- return new TestTypedAdapter(tupleParserFactory, adapterOutputType, ctx, configuration);
+ ITupleParserFactory tupleParserFactory = new AsterixTupleParserFactory(configuration, outputType,
+ InputDataFormat.ADM);
+ return new TestTypedAdapter(tupleParserFactory, outputType, ctx, configuration, partition);
}
@Override
public ARecordType getAdapterOutputType() {
- return adapterOutputType;
+ return outputType;
}
@Override
- public void configure(Map<String, String> configuration) throws Exception {
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
this.configuration = configuration;
+ this.outputType = outputType;
+ }
+
+ @Override
+ public boolean isRecordTrackingEnabled() {
+ return false;
+ }
+
+ @Override
+ public IIntakeProgressTracker createIntakeProgressTracker() {
+ return null;
}
}
diff --git a/asterix-external-data/src/test/resources/text_functions.xml b/asterix-external-data/src/test/resources/library_descriptor.xml
similarity index 75%
rename from asterix-external-data/src/test/resources/text_functions.xml
rename to asterix-external-data/src/test/resources/library_descriptor.xml
index 8c7a92c..e35288f 100644
--- a/asterix-external-data/src/test/resources/text_functions.xml
+++ b/asterix-external-data/src/test/resources/library_descriptor.xml
@@ -11,6 +11,22 @@
</libraryFunction>
<libraryFunction>
<function_type>SCALAR</function_type>
+ <name>addHashTags</name>
+ <arguments>Tweet</arguments>
+ <return_type>ProcessedTweet</return_type>
+ <definition>edu.uci.ics.asterix.external.library.AddHashTagsFactory
+ </definition>
+ </libraryFunction>
+ <libraryFunction>
+ <function_type>SCALAR</function_type>
+ <name>addHashTagsInPlace</name>
+ <arguments>Tweet</arguments>
+ <return_type>ProcessedTweet</return_type>
+ <definition>edu.uci.ics.asterix.external.library.AddHashTagsInPlaceFactory
+ </definition>
+ </libraryFunction>
+ <libraryFunction>
+ <function_type>SCALAR</function_type>
<name>mysum</name>
<arguments>AINT32,AINT32</arguments>
<return_type>AINT32</return_type>
@@ -53,7 +69,8 @@
<libraryAdapters>
<libraryAdapter>
<name>test_typed_adapter</name>
- <factory_class>edu.uci.ics.asterix.external.library.adapter.TestTypedAdapterFactory</factory_class>
+ <factory_class>edu.uci.ics.asterix.external.library.adapter.TestTypedAdapterFactory
+ </factory_class>
</libraryAdapter>
</libraryAdapters>
</externalLibrary>
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
deleted file mode 100644
index c532de5..0000000
--- a/asterix-installer/src/test/java/edu/uci/ics/asterix/installer/test/AsterixFaultToleranceIT.java
+++ /dev/null
@@ -1,82 +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.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, null, false);
- }
- }
-
- 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/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
deleted file mode 100644
index 1f0678e..0000000
--- a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.1.ddl.aql
+++ /dev/null
@@ -1,28 +0,0 @@
-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: int64,
- 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"="30"),("tps"="50"),("tput-duration"="5"),("mode"="controlled"));
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
deleted file mode 100644
index 64dbf25..0000000
--- a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.2.update.aql
+++ /dev/null
@@ -1,3 +0,0 @@
-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.sleep.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.3.sleep.aql
deleted file mode 100644
index 5caff40..0000000
--- a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.3.sleep.aql
+++ /dev/null
@@ -1 +0,0 @@
-10000
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
deleted file mode 100644
index 2d8a23e..0000000
--- a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.4.mgx.aql
+++ /dev/null
@@ -1 +0,0 @@
-stop -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.5.mgx.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.5.mgx.aql
deleted file mode 100644
index 4e99f33..0000000
--- a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.5.mgx.aql
+++ /dev/null
@@ -1 +0,0 @@
-start -n asterix
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.6.sleep.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.6.sleep.aql
deleted file mode 100644
index c5da56a..0000000
--- a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.6.sleep.aql
+++ /dev/null
@@ -1 +0,0 @@
-40000
diff --git a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.7.query.aql b/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.7.query.aql
deleted file mode 100644
index d03b9fe..0000000
--- a/asterix-installer/src/test/resources/integrationts/fault-tolerance/queries/feeds/IN1-cluster-restart/in1-cluster-restart.7.query.aql
+++ /dev/null
@@ -1,10 +0,0 @@
-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.2.adm b/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.2.adm
deleted file mode 100644
index 3c13d5f..0000000
--- a/asterix-installer/src/test/resources/integrationts/fault-tolerance/results/feeds/IN1-cluster-restart/IN1-cluster-restart.2.adm
+++ /dev/null
@@ -1,2 +0,0 @@
-[ 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
deleted file mode 100644
index 0d9ed23..0000000
--- a/asterix-installer/src/test/resources/integrationts/fault-tolerance/testsuite.xml
+++ /dev/null
@@ -1,10 +0,0 @@
-<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/results/library-metadata/functionDataset/functionDataset.1.adm b/asterix-installer/src/test/resources/integrationts/library/results/library-metadata/functionDataset/functionDataset.1.adm
index b3bf441..a9bcf09 100644
--- 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
@@ -1,7 +1,9 @@
-[ { "DataverseName": "externallibtest", "Name": "testlib#allTypes", "Arity": "1", "Params": [ "AllType" ], "ReturnType": "AllType", "Definition": "edu.uci.ics.asterix.external.library.AllTypesFactory\n\t\t\t", "Language": "JAVA", "Kind": "SCALAR" }
-, { "DataverseName": "externallibtest", "Name": "testlib#echoDelay", "Arity": "1", "Params": [ "TweetMessageType" ], "ReturnType": "TweetMessageType", "Definition": "edu.uci.ics.asterix.external.library.EchoDelayFactory\n\t\t\t", "Language": "JAVA", "Kind": "SCALAR" }
-, { "DataverseName": "externallibtest", "Name": "testlib#getCapital", "Arity": "1", "Params": [ "ASTRING" ], "ReturnType": "CountryCapitalType", "Definition": "edu.uci.ics.asterix.external.library.CapitalFinderFactory\n\t\t\t", "Language": "JAVA", "Kind": "SCALAR" }
-, { "DataverseName": "externallibtest", "Name": "testlib#mysum", "Arity": "2", "Params": [ "AINT32", "AINT32" ], "ReturnType": "AINT32", "Definition": "edu.uci.ics.asterix.external.library.SumFactory\n\t\t\t", "Language": "JAVA", "Kind": "SCALAR" }
-, { "DataverseName": "externallibtest", "Name": "testlib#parseTweet", "Arity": "1", "Params": [ "TweetInputType" ], "ReturnType": "TweetOutputType", "Definition": "edu.uci.ics.asterix.external.library.ParseTweetFactory\n\t\t\t", "Language": "JAVA", "Kind": "SCALAR" }
-, { "DataverseName": "externallibtest", "Name": "testlib#toUpper", "Arity": "1", "Params": [ "TextType" ], "ReturnType": "TextType", "Definition": "edu.uci.ics.asterix.external.library.UpperCaseFactory\n\t\t\t", "Language": "JAVA", "Kind": "SCALAR" }
+[ { "DataverseName": "externallibtest", "Name": "testlib#addHashTags", "Arity": "1", "Params": [ "Tweet" ], "ReturnType": "ProcessedTweet", "Definition": "edu.uci.ics.asterix.external.library.AddHashTagsFactory", "Language": "JAVA", "Kind": "SCALAR" }
+, { "DataverseName": "externallibtest", "Name": "testlib#addHashTagsInPlace", "Arity": "1", "Params": [ "Tweet" ], "ReturnType": "ProcessedTweet", "Definition": "edu.uci.ics.asterix.external.library.AddHashTagsInPlaceFactory", "Language": "JAVA", "Kind": "SCALAR" }
+, { "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": [ "TweetInputType" ], "ReturnType": "TweetOutputType", "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/testsuite.xml b/asterix-installer/src/test/resources/integrationts/library/testsuite.xml
index ef21a16..be9ba0e 100644
--- a/asterix-installer/src/test/resources/integrationts/library/testsuite.xml
+++ b/asterix-installer/src/test/resources/integrationts/library/testsuite.xml
@@ -39,7 +39,7 @@
</test-case>
</test-group>
<test-group name="library-feeds">
- <test-case FilePath="library-feeds" category="slow">
+ <test-case FilePath="library-feeds">
<compilation-unit name="feed_ingest">
<output-dir compare="Text">feed_ingest</output-dir>
</compilation-unit>
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 6eb60f0..12e50dc 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
@@ -23,7 +23,6 @@
import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
-import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
import edu.uci.ics.asterix.metadata.entities.CompactionPolicy;
@@ -32,7 +31,6 @@
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;
@@ -64,9 +62,7 @@
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
@@ -110,7 +106,6 @@
synchronized (datatypes) {
synchronized (functions) {
synchronized (adapters) {
- synchronized (feedActivity) {
synchronized (libraries) {
synchronized (compactionPolicies) {
dataverses.clear();
@@ -120,7 +115,6 @@
datatypes.clear();
functions.clear();
adapters.clear();
- feedActivity.clear();
libraries.clear();
compactionPolicies.clear();
}
@@ -133,7 +127,7 @@
}
}
}
- }
+
public Object addDataverseIfNotExists(Dataverse dataverse) {
synchronized (dataverses) {
@@ -240,7 +234,6 @@
synchronized (functions) {
synchronized (adapters) {
synchronized (libraries) {
- synchronized (feedActivity) {
synchronized (feeds) {
synchronized (compactionPolicies) {
datasets.remove(dataverse.getDataverseName());
@@ -257,19 +250,8 @@
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());
}
}
@@ -281,7 +263,7 @@
}
}
}
- }
+
public Object dropDataset(Dataset dataset) {
synchronized (datasets) {
@@ -503,9 +485,9 @@
adaptersInDataverse = new HashMap<String, DatasourceAdapter>();
adapters.put(adapter.getAdapterIdentifier().getNamespace(), adaptersInDataverse);
}
- DatasourceAdapter adapterObject = adaptersInDataverse.get(adapter.getAdapterIdentifier().getAdapterName());
+ DatasourceAdapter adapterObject = adaptersInDataverse.get(adapter.getAdapterIdentifier().getName());
if (adapterObject == null) {
- return adaptersInDataverse.put(adapter.getAdapterIdentifier().getAdapterName(), adapter);
+ return adaptersInDataverse.put(adapter.getAdapterIdentifier().getName(), adapter);
}
return null;
}
@@ -516,28 +498,14 @@
Map<String, DatasourceAdapter> adaptersInDataverse = adapters.get(adapter.getAdapterIdentifier()
.getNamespace());
if (adaptersInDataverse != null) {
- return adaptersInDataverse.remove(adapter.getAdapterIdentifier().getAdapterName());
+ return adaptersInDataverse.remove(adapter.getAdapterIdentifier().getName());
}
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) {
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 dbfe32c..4842934 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
@@ -24,7 +24,6 @@
import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
@@ -37,8 +36,6 @@
import edu.uci.ics.asterix.metadata.entities.Dataverse;
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
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;
@@ -717,29 +714,7 @@
}
return adapter;
}
-
- @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;
- }
+
@Override
public void dropLibrary(MetadataTransactionContext ctx, String dataverseName, String libraryName)
@@ -822,18 +797,7 @@
}
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 {
@@ -848,12 +812,14 @@
@Override
public void dropFeed(MetadataTransactionContext ctx, String dataverse, String feedName) throws MetadataException {
+ Feed feed = null;
try {
+ feed = metadataNode.getFeed(ctx.getJobId(), dataverse, feedName);
metadataNode.dropFeed(ctx.getJobId(), dataverse, feedName);
} catch (RemoteException e) {
throw new MetadataException(e);
}
- ctx.dropFeed(dataverse, feedName);
+ ctx.dropFeed(feed);
}
@Override
@@ -866,18 +832,7 @@
ctx.addFeed(feed);
}
- @Override
- 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;
- }
-
+
public List<DatasourceAdapter> getDataverseAdapters(MetadataTransactionContext mdTxnCtx, String dataverse)
throws MetadataException {
List<DatasourceAdapter> dataverseAdapters;
@@ -888,6 +843,30 @@
}
return dataverseAdapters;
}
+
+ public void dropFeedPolicy(MetadataTransactionContext mdTxnCtx, String dataverseName, String policyName)
+ throws MetadataException {
+ FeedPolicy feedPolicy = null;
+ try {
+ feedPolicy = metadataNode.getFeedPolicy(mdTxnCtx.getJobId(), dataverseName, policyName);
+ metadataNode.dropFeedPolicy(mdTxnCtx.getJobId(), dataverseName, policyName);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ mdTxnCtx.dropFeedPolicy(feedPolicy);
+ }
+
+ public List<FeedPolicy> getDataversePolicies(MetadataTransactionContext mdTxnCtx, String dataverse)
+ throws MetadataException {
+ List<FeedPolicy> dataverseFeedPolicies;
+ try {
+ dataverseFeedPolicies = metadataNode.getDataversePolicies(mdTxnCtx.getJobId(), dataverse);
+ } catch (RemoteException e) {
+ throw new MetadataException(e);
+ }
+ return dataverseFeedPolicies;
+ }
+
@Override
public List<ExternalFile> getDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset)
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 a593348..0ff5455 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,19 +17,13 @@
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;
import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
import edu.uci.ics.asterix.common.transactions.DatasetId;
@@ -51,8 +45,6 @@
import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
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;
@@ -66,7 +58,6 @@
import edu.uci.ics.asterix.metadata.entitytupletranslators.DatatypeTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.DataverseTupleTranslator;
import edu.uci.ics.asterix.metadata.entitytupletranslators.ExternalFileTupleTranslator;
-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;
@@ -74,7 +65,6 @@
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.valueextractors.DatasetNameValueExtractor;
import edu.uci.ics.asterix.metadata.valueextractors.DatatypeNameValueExtractor;
import edu.uci.ics.asterix.metadata.valueextractors.MetadataEntityValueExtractor;
@@ -374,7 +364,7 @@
if (dataverseAdapters != null && dataverseAdapters.size() > 0) {
// Drop all functions in this dataverse.
for (DatasourceAdapter adapter : dataverseAdapters) {
- dropAdapter(jobId, dataverseName, adapter.getAdapterIdentifier().getAdapterName());
+ dropAdapter(jobId, dataverseName, adapter.getAdapterIdentifier().getName());
}
}
@@ -388,6 +378,14 @@
dropFeed(jobId, dataverseName, feed.getFeedName());
}
}
+
+ List<FeedPolicy> feedPolicies = getDataversePolicies(jobId, dataverseName);
+ if (feedPolicies != null && feedPolicies.size() > 0) {
+ // Drop all feed ingestion policies in this dataverse.
+ for (FeedPolicy feedPolicy : feedPolicies) {
+ dropFeedPolicy(jobId, dataverseName, feedPolicy.getPolicyName());
+ }
+ }
// Delete the dataverse entry from the 'dataverse' dataset.
ITupleReference searchKey = createTuple(dataverseName);
@@ -1152,7 +1150,7 @@
insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
} catch (TreeIndexDuplicateKeyException e) {
- throw new MetadataException("A adapter with this name " + adapter.getAdapterIdentifier().getAdapterName()
+ throw new MetadataException("A adapter with this name " + adapter.getAdapterIdentifier().getName()
+ " already exists in dataverse '" + adapter.getAdapterIdentifier().getNamespace() + "'.", e);
} catch (Exception e) {
throw new MetadataException(e);
@@ -1333,76 +1331,7 @@
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 {
@@ -1439,57 +1368,7 @@
}
}
- @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 {
@@ -1543,46 +1422,37 @@
}
- public List<FeedActivity> getDatasetsServedByFeed(JobId jobId, String dataverse, String feedName)
- throws MetadataException, RemoteException {
- List<FeedActivity> feedActivities = new ArrayList<FeedActivity>();
+
+ @Override
+ public void dropFeedPolicy(JobId jobId, String dataverseName, String policyName) throws MetadataException,
+ RemoteException {
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;
+ ITupleReference searchKey = createTuple(dataverseName, policyName);
+ ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey);
+ deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, tuple);
+ } catch (TreeIndexException e) {
+ throw new MetadataException("Unknown feed policy " + policyName, e);
} catch (Exception e) {
throw new MetadataException(e);
}
}
@Override
+ public List<FeedPolicy> getDataversePolicies(JobId jobId, String dataverse) throws MetadataException,
+ RemoteException {
+ try {
+ ITupleReference searchKey = createTuple(dataverse);
+ FeedPolicyTupleTranslator tupleReaderWriter = new FeedPolicyTupleTranslator(false);
+ IValueExtractor<FeedPolicy> valueExtractor = new MetadataEntityValueExtractor<FeedPolicy>(tupleReaderWriter);
+ List<FeedPolicy> results = new ArrayList<FeedPolicy>();
+ searchIndex(jobId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey, valueExtractor, results);
+ return results;
+ } catch (Exception e) {
+ throw new MetadataException(e);
+ }
+ }
+
+ @Override
public void addExternalFile(JobId jobId, ExternalFile externalFile) throws MetadataException, RemoteException {
try {
// Insert into the 'externalFiles' dataset.
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 826ebbf..24962b5 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
@@ -26,11 +26,14 @@
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.Feed.FeedType;
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.entities.PrimaryFeed;
+import edu.uci.ics.asterix.metadata.entities.SecondaryFeed;
import edu.uci.ics.asterix.metadata.feeds.AdapterIdentifier;
/**
@@ -228,8 +231,16 @@
}
- public void dropFeed(String dataverse, String feedName) {
- Feed feed = new Feed(dataverse, feedName, null, null, null);
+ public void dropFeed(String dataverseName, String feedName, FeedType feedType) {
+ Feed feed = null;
+ switch (feedType) {
+ case PRIMARY:
+ feed = new PrimaryFeed(dataverseName, feedName, null, null, null);
+ break;
+ case SECONDARY:
+ feed = new SecondaryFeed(dataverseName, feedName, null, null);
+ break;
+ }
droppedCache.addFeedIfNotExists(feed);
logAndApply(new MetadataLogicalOperation(feed, false));
}
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
index ea07a62..0351468 100644
--- 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
@@ -2,6 +2,7 @@
import java.util.Set;
+import edu.uci.ics.asterix.common.api.IClusterEventsSubscriber;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.event.schema.cluster.Node;
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 0b3bae6..6c098fb 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
@@ -19,7 +19,6 @@
import java.util.List;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.metadata.MetadataException;
import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
@@ -30,8 +29,6 @@
import edu.uci.ics.asterix.metadata.entities.Dataverse;
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
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;
@@ -489,26 +486,8 @@
*/
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
@@ -525,19 +504,7 @@
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;
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 56fbcb5..ec727a9 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
@@ -21,7 +21,6 @@
import java.util.List;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.common.transactions.JobId;
import edu.uci.ics.asterix.metadata.MetadataException;
@@ -32,8 +31,6 @@
import edu.uci.ics.asterix.metadata.entities.Dataverse;
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
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;
@@ -497,16 +494,6 @@
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
@@ -559,16 +546,7 @@
*/
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
@@ -588,17 +566,7 @@
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.
@@ -667,16 +635,28 @@
public List<Feed> getDataverseFeeds(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
/**
+ * delete a give feed (ingestion) policy
+ *
* @param jobId
* @param dataverseName
- * @param deedName
+ * @param policyName
+ * @return
+ * @throws RemoteException
+ * @throws MetadataException
+ */
+ public void dropFeedPolicy(JobId jobId, String dataverseName, String policyName) throws MetadataException,
+ RemoteException;
+
+ /**
+ * @param jobId
+ * @param dataverse
* @return
* @throws MetadataException
* @throws RemoteException
*/
- public List<FeedActivity> getDatasetsServedByFeed(JobId jobId, String dataverseName, String deedName)
- throws MetadataException, RemoteException;
-
+ public List<FeedPolicy> getDataversePolicies(JobId jobId, String dataverse) throws MetadataException,
+ RemoteException;
+
/**
* @param jobId
* A globally unique id for an active metadata transaction.
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 c93c29e..0ac211e 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
@@ -57,9 +57,9 @@
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.external.IAdapterFactory;
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;
@@ -123,7 +123,7 @@
MetadataPrimaryIndexes.INDEX_DATASET, MetadataPrimaryIndexes.NODE_DATASET,
MetadataPrimaryIndexes.NODEGROUP_DATASET, MetadataPrimaryIndexes.FUNCTION_DATASET,
MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.FEED_DATASET,
- MetadataPrimaryIndexes.FEED_ACTIVITY_DATASET, MetadataPrimaryIndexes.FEED_POLICY_DATASET,
+ MetadataPrimaryIndexes.FEED_POLICY_DATASET,
MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET,
MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET };
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 7eae5cd..04e56b1 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
@@ -15,7 +15,6 @@
package edu.uci.ics.asterix.metadata.bootstrap;
-import java.util.ArrayList;
import java.util.Arrays;
import edu.uci.ics.asterix.metadata.MetadataException;
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 30ae9fa..b726ed6 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
@@ -50,19 +50,21 @@
public static ARecordType FUNCTION_RECORDTYPE;
public static ARecordType DATASOURCE_ADAPTER_RECORDTYPE;
public static ARecordType FEED_RECORDTYPE;
+ public static ARecordType PRIMARY_FEED_DETAILS_RECORDTYPE;
+ public static ARecordType SECONDARY_FEED_DETAILS_RECORDTYPE;
public static ARecordType FEED_ADAPTER_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;
public static ARecordType EXTERNAL_FILE_RECORDTYPE;
/**
* Create all metadata record types.
+ * @throws HyracksDataException
*/
- public static void init() throws MetadataException {
+ public static void init() throws MetadataException, HyracksDataException {
// Attention: The order of these calls is important because some types
// depend on other types being created first.
// These calls are one "dependency chain".
@@ -90,10 +92,10 @@
FUNCTION_RECORDTYPE = createFunctionRecordType();
DATASOURCE_ADAPTER_RECORDTYPE = createDatasourceAdapterRecordType();
- FEED_RECORDTYPE = createFeedRecordType();
FEED_ADAPTER_CONFIGURATION_RECORDTYPE = createPropertiesRecordType();
- FEED_ACTIVITY_DETAILS_RECORDTYPE = createPropertiesRecordType();
- FEED_ACTIVITY_RECORDTYPE = createFeedActivityRecordType();
+ PRIMARY_FEED_DETAILS_RECORDTYPE = createPrimaryFeedDetailsRecordType();
+ SECONDARY_FEED_DETAILS_RECORDTYPE = createSecondaryFeedDetailsRecordType();
+ FEED_RECORDTYPE = createFeedRecordType();
FEED_POLICY_RECORDTYPE = createFeedPolicyRecordType();
LIBRARY_RECORDTYPE = createLibraryRecordType();
@@ -498,50 +500,67 @@
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 };
- try {
- return new ARecordType("FeedActivityRecordType", fieldNames, fieldTypes, true);
- } catch (HyracksDataException e) {
- throw new AsterixException(e);
- }
- }
+
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_ADAPTER_NAME_FIELD_INDEX = 2;
- public static final int FEED_ARECORD_ADAPTER_CONFIGURATION_FIELD_INDEX = 3;
- public static final int FEED_ARECORD_FUNCTION_FIELD_INDEX = 4;
- public static final int FEED_ARECORD_TIMESTAMP_FIELD_INDEX = 5;
+ public static final int FEED_ARECORD_FUNCTION_FIELD_INDEX = 2;
+ public static final int FEED_ARECORD_FEED_TYPE_FIELD_INDEX = 3;
+ public static final int FEED_ARECORD_PRIMARY_TYPE_DETAILS_FIELD_INDEX = 4;
+ public static final int FEED_ARECORD_SECONDARY_TYPE_DETAILS_FIELD_INDEX = 5;
+ public static final int FEED_ARECORD_TIMESTAMP_FIELD_INDEX = 6;
- private static ARecordType createFeedRecordType() throws AsterixException {
+
+ public static final int FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_NAME_FIELD_INDEX = 0;
+ public static final int FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_CONFIGURATION_FIELD_INDEX = 1;
- AUnorderedListType unorderedAdapterPropertyListType = new AUnorderedListType(
- DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE, null);
+ public static final int FEED_ARECORD_SECONDARY_FIELD_DETAILS_SOURCE_FEED_NAME_FIELD_INDEX = 0;
+
+ private static ARecordType createFeedRecordType() throws AsterixException, HyracksDataException {
List<IAType> feedFunctionUnionList = new ArrayList<IAType>();
feedFunctionUnionList.add(BuiltinType.ANULL);
feedFunctionUnionList.add(BuiltinType.ASTRING);
AUnionType feedFunctionUnion = new AUnionType(feedFunctionUnionList, null);
- String[] fieldNames = { "DataverseName", "FeedName", "AdapterName", "AdapterConfiguration", "Function",
- "Timestamp" };
- IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
- unorderedAdapterPropertyListType, feedFunctionUnion, BuiltinType.ASTRING };
+ List<IAType> primaryFeedTypeDetailsRecordUnionList = new ArrayList<IAType>();
+ primaryFeedTypeDetailsRecordUnionList.add(BuiltinType.ANULL);
+ primaryFeedTypeDetailsRecordUnionList.add(PRIMARY_FEED_DETAILS_RECORDTYPE);
+ AUnionType primaryRecordUnion = new AUnionType(primaryFeedTypeDetailsRecordUnionList, null);
- try {
- return new ARecordType("FeedRecordType", fieldNames, fieldTypes, true);
- } catch (HyracksDataException e) {
- throw new AsterixException(e);
- }
+ List<IAType> secondaryFeedTypeDetailsRecordUnionList = new ArrayList<IAType>();
+ secondaryFeedTypeDetailsRecordUnionList.add(BuiltinType.ANULL);
+ secondaryFeedTypeDetailsRecordUnionList.add(SECONDARY_FEED_DETAILS_RECORDTYPE);
+ AUnionType secondaryRecordUnion = new AUnionType(secondaryFeedTypeDetailsRecordUnionList, null);
+ String[] fieldNames = { "DataverseName", "FeedName", "Function", "FeedType", "PrimaryTypeDetails",
+ "SecondaryTypeDetails", "Timestamp" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, feedFunctionUnion, BuiltinType.ASTRING,
+ primaryRecordUnion, secondaryRecordUnion, BuiltinType.ASTRING };
+
+ return new ARecordType("FeedRecordType", fieldNames, fieldTypes, true);
}
+ public static final int FEED_TYPE_PRIMARY_ARECORD_ADAPTER_NAME_FIELD_INDEX = 0;
+ public static final int FEED_TYPE_PRIMARY_ARECORD_ADAPTER_CONFIGURATION_FIELD_INDEX = 1;
+
+ private static final ARecordType createPrimaryFeedDetailsRecordType() throws AsterixException, HyracksDataException {
+ AUnorderedListType unorderedAdaptorPropertyListType = new AUnorderedListType(
+ DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE, null);
+
+ String[] fieldNames = { "AdapterName", "AdapterConfiguration" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING, unorderedAdaptorPropertyListType };
+ return new ARecordType(null, fieldNames, fieldTypes, true);
+ }
+
+ public static final int FEED_TYPE_SECONDARY_ARECORD_SOURCE_FEED_NAME_FIELD_INDEX = 0;
+
+ private static final ARecordType createSecondaryFeedDetailsRecordType() throws AsterixException, HyracksDataException {
+ String[] fieldNames = { "SourceFeedName" };
+ IAType[] fieldTypes = { BuiltinType.ASTRING };
+ return new ARecordType(null, 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;
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
index 6948dbc..037b04d 100644
--- 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
@@ -16,8 +16,8 @@
import java.util.concurrent.atomic.AtomicInteger;
-import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
-import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+import edu.uci.ics.asterix.common.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.common.api.IClusterManagementWork;
public abstract class AbstractClusterManagementWork implements IClusterManagementWork {
@@ -35,6 +35,8 @@
this.workId = WorkIdGenerator.getNextWorkId();
}
+
+
private static class WorkIdGenerator {
private static AtomicInteger workId = new AtomicInteger(0);
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
index 68dcc4c..1157562 100644
--- 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
@@ -1,27 +1,49 @@
+/*
+ * 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 edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+import java.util.Set;
+
+import edu.uci.ics.asterix.common.api.IClusterEventsSubscriber;
public class AddNodeWork extends AbstractClusterManagementWork {
- private final int numberOfNodes;
+ private final int numberOfNodesRequested;
+ private final Set<String> deadNodes;
@Override
public WorkType getClusterManagementWorkType() {
return WorkType.ADD_NODE;
}
- public AddNodeWork(int numberOfNodes, IClusterEventsSubscriber subscriber) {
+ public AddNodeWork(Set<String> deadNodes, int numberOfNodesRequested, IClusterEventsSubscriber subscriber) {
super(subscriber);
- this.numberOfNodes = numberOfNodes;
+ this.deadNodes = deadNodes;
+ this.numberOfNodesRequested = numberOfNodesRequested;
}
- public int getNumberOfNodes() {
- return numberOfNodes;
+ public int getNumberOfNodesRequested() {
+ return numberOfNodesRequested;
+ }
+
+ public Set<String> getDeadNodes() {
+ return deadNodes;
}
@Override
public String toString() {
- return WorkType.ADD_NODE + " " + numberOfNodes + " requested by " + subscriber;
+ return WorkType.ADD_NODE + " " + numberOfNodesRequested + " requested by " + subscriber;
}
}
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
index d578a77..7f3b575 100644
--- 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
@@ -1,6 +1,7 @@
package edu.uci.ics.asterix.metadata.cluster;
-import edu.uci.ics.asterix.metadata.api.IClusterManagementWork;
+import edu.uci.ics.asterix.common.api.IClusterManagementWork;
+import edu.uci.ics.asterix.common.api.IClusterManagementWorkResponse;
public class ClusterManagementWorkResponse implements IClusterManagementWorkResponse {
@@ -13,6 +14,7 @@
this.status = Status.IN_PROGRESS;
}
+
@Override
public IClusterManagementWork getWork() {
return work;
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
index fe7f4a4..47ca953 100644
--- 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
@@ -25,6 +25,7 @@
import javax.xml.bind.JAXBContext;
import javax.xml.bind.Unmarshaller;
+import edu.uci.ics.asterix.common.api.IClusterEventsSubscriber;
import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.event.management.AsterixEventServiceClient;
@@ -39,7 +40,6 @@
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;
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
index 90683d1..8b4f2aa 100644
--- 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
@@ -2,7 +2,7 @@
import java.util.Set;
-import edu.uci.ics.asterix.metadata.api.IClusterEventsSubscriber;
+import edu.uci.ics.asterix.common.api.IClusterEventsSubscriber;
public class RemoveNodeWork extends AbstractClusterManagementWork {
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 ef23135..5df2c95 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
@@ -43,8 +43,7 @@
public abstract class AqlDataSource implements IDataSource<AqlSourceId> {
private final AqlSourceId id;
- private final String datasourceDataverse;
- private final String datasourceName;
+ private final IAType itemType;
private final AqlDataSourceType datasourceType;
protected IAType[] schemaTypes;
protected INodeDomain domain;
@@ -58,19 +57,18 @@
}
public AqlDataSource(AqlSourceId id, String datasourceDataverse, String datasourceName,
- AqlDataSourceType datasourceType) throws AlgebricksException {
+ IAType itemType, AqlDataSourceType datasourceType) throws AlgebricksException {
this.id = id;
- this.datasourceDataverse = datasourceDataverse;
- this.datasourceName = datasourceName;
+ this.itemType = itemType;
this.datasourceType = datasourceType;
}
public String getDatasourceDataverse() {
- return datasourceDataverse;
+ return id.getDataverseName();
}
public String getDatasourceName() {
- return datasourceName;
+ return id.getDatasourceName();
}
@Override
@@ -196,7 +194,10 @@
public Map<String, Serializable> getProperties() {
return properties;
}
-
+
+ public IAType getItemType() {
+ return itemType;
+ }
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 5055cea..0ec098d 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
@@ -18,6 +18,7 @@
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
@@ -37,7 +38,12 @@
import edu.uci.ics.asterix.common.dataflow.AsterixLSMTreeInsertDeleteOperatorDescriptor;
import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedActivity;
+import edu.uci.ics.asterix.common.feeds.FeedActivity.FeedActivityDetails;
import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.feeds.api.ICentralFeedManager;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
@@ -65,23 +71,19 @@
import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
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.entities.PrimaryFeed;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory.SupportedOperation;
import edu.uci.ics.asterix.metadata.external.IndexingConstants;
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.FeedCollectOperatorDescriptor;
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.feeds.IFeedAdapterFactory;
import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
import edu.uci.ics.asterix.metadata.utils.ExternalDatasetsRegistry;
import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
@@ -187,6 +189,7 @@
private boolean asyncResults;
private ResultSetId resultSetId;
private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
+ private final ICentralFeedManager centralFeedManager;
private final Dataverse defaultDataverse;
private JobId jobId;
@@ -213,10 +216,11 @@
return config;
}
- public AqlMetadataProvider(Dataverse defaultDataverse) {
+ public AqlMetadataProvider(Dataverse defaultDataverse, ICentralFeedManager centralFeedManager) {
this.defaultDataverse = defaultDataverse;
this.stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
this.storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
+ this.centralFeedManager = centralFeedManager;
}
public void setJobId(JobId jobId) {
@@ -330,10 +334,8 @@
JobGenContext context, JobSpecification jobSpec, Object implConfig) throws AlgebricksException {
try {
switch (((AqlDataSource) dataSource).getDatasourceType()) {
- case FEED: {
- // loading data from a feed
- return buildFeedIntakeRuntime(jobSpec, dataSource);
- }
+ case FEED:
+ return buildFeedCollectRuntime(jobSpec, dataSource);
case INTERNAL_DATASET: {
// querying an internal dataset
return buildInternalDatasetScan(jobSpec, scanVariables, minFilterVars, maxFilterVars, opSchema,
@@ -376,6 +378,110 @@
}
}
+@SuppressWarnings("rawtypes")
+public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedCollectRuntime(JobSpecification jobSpec,
+ IDataSource<AqlSourceId> dataSource) throws AlgebricksException {
+
+ FeedDataSource feedDataSource = (FeedDataSource) dataSource;
+ FeedCollectOperatorDescriptor feedCollector = null;
+
+ try {
+ ARecordType feedOutputType = (ARecordType) feedDataSource.getItemType();
+ ISerializerDeserializer payloadSerde = NonTaggedDataFormat.INSTANCE.getSerdeProvider()
+ .getSerializerDeserializer(feedOutputType);
+ RecordDescriptor feedDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
+
+ FeedPolicy feedPolicy = (FeedPolicy) ((AqlDataSource) dataSource).getProperties().get(
+ BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY);
+ if (feedPolicy == null) {
+ throw new AlgebricksException("Feed not configured with a policy");
+ }
+ feedPolicy.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy.getPolicyName());
+ FeedConnectionId feedConnectionId = new FeedConnectionId(feedDataSource.getId().getDataverseName(),
+ feedDataSource.getId().getDatasourceName(), feedDataSource.getTargetDataset());
+ feedCollector = new FeedCollectOperatorDescriptor(jobSpec, feedConnectionId,
+ feedDataSource.getSourceFeedId(), (ARecordType) feedOutputType, feedDesc,
+ feedPolicy.getProperties(), feedDataSource.getLocation());
+
+ return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedCollector,
+ determineLocationConstraint(feedDataSource));
+
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+}
+
+private AlgebricksAbsolutePartitionConstraint determineLocationConstraint(FeedDataSource feedDataSource)
+ throws AsterixException {
+ String[] locationArray = null;
+ String locations = null;;
+ switch (feedDataSource.getSourceFeedType()) {
+ case PRIMARY:
+ switch (feedDataSource.getLocation()) {
+ case SOURCE_FEED_COMPUTE_STAGE:
+ if (feedDataSource.getFeed().getFeedId().equals(feedDataSource.getSourceFeedId())) {
+ locationArray = feedDataSource.getLocations();
+ } else {
+ Collection<FeedActivity> activities = centralFeedManager.getFeedLoadManager()
+ .getFeedActivities();
+ Iterator<FeedActivity> it = activities.iterator();
+ FeedActivity activity = null;
+ while (it.hasNext()) {
+ activity = it.next();
+ if (activity.getDataverseName().equals(feedDataSource.getSourceFeedId().getDataverse())
+ && activity.getFeedName()
+ .equals(feedDataSource.getSourceFeedId().getFeedName())) {
+ locations = activity.getFeedActivityDetails().get(
+ FeedActivityDetails.COMPUTE_LOCATIONS);
+ locationArray = locations.split(",");
+ break;
+ }
+ }
+ }
+ break;
+ case SOURCE_FEED_INTAKE_STAGE:
+ locationArray = feedDataSource.getLocations();
+ break;
+ }
+ break;
+ case SECONDARY:
+ Collection<FeedActivity> activities = centralFeedManager.getFeedLoadManager().getFeedActivities();
+ Iterator<FeedActivity> it = activities.iterator();
+ FeedActivity activity = null;
+ while (it.hasNext()) {
+ activity = it.next();
+ if (activity.getDataverseName().equals(feedDataSource.getSourceFeedId().getDataverse())
+ && activity.getFeedName().equals(feedDataSource.getSourceFeedId().getFeedName())) {
+ switch (feedDataSource.getLocation()) {
+ case SOURCE_FEED_INTAKE_STAGE:
+ locations = activity.getFeedActivityDetails()
+ .get(FeedActivityDetails.COLLECT_LOCATIONS);
+ break;
+ case SOURCE_FEED_COMPUTE_STAGE:
+ locations = activity.getFeedActivityDetails()
+ .get(FeedActivityDetails.COMPUTE_LOCATIONS);
+ break;
+ }
+ break;
+ }
+ }
+
+ if (locations != null) {
+ locationArray = locations.split(",");
+ } else {
+ String message = "Unable to discover location(s) for source feed data hand-off "
+ + feedDataSource.getSourceFeedId();
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe(message);
+ }
+ throw new AsterixException(message);
+ }
+ break;
+ }
+ AlgebricksAbsolutePartitionConstraint locationConstraint = new AlgebricksAbsolutePartitionConstraint(
+ locationArray);
+ return locationConstraint;
+}
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildLoadableDatasetScan(JobSpecification jobSpec,
LoadableDataSource alds, IAdapterFactory adapterFactory, RecordDescriptor rDesc, boolean isPKAutoGenerated,
List<List<String>> primaryKeys, ARecordType recType, int pkIndex) throws AlgebricksException {
@@ -411,7 +517,7 @@
JobGenContext context, Object implConfig) throws AlgebricksException, MetadataException {
AqlSourceId asid = dataSource.getId();
String dataverseName = asid.getDataverseName();
- String datasetName = asid.getDatasetName();
+ String datasetName = asid.getDatasourceName();
Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
int[] minFilterFieldIndexes = null;
@@ -439,8 +545,8 @@
}
private IAdapterFactory getConfiguredAdapterFactory(Dataset dataset, String adapterName,
- Map<String, String> configuration, IAType itemType, boolean isPKAutoGenerated,
- List<List<String>> primaryKeys) throws AlgebricksException {
+ Map<String, String> configuration, IAType itemType, boolean isPKAutoGenerated, List<List<String>> primaryKeys)
+ throws AlgebricksException {
IAdapterFactory adapterFactory;
DatasourceAdapter adapterEntity;
String adapterFactoryClassname;
@@ -458,6 +564,8 @@
adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
}
+ adapterFactory.configure(configuration, (ARecordType) itemType);
+
// check to see if dataset is indexed
Index filesIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(),
@@ -472,18 +580,11 @@
iterator.remove();
}
}
- ((IGenericAdapterFactory) adapterFactory).setFiles(files);
+ // TODO Check this call, result of merge from master!
+ // ((IGenericAdapterFactory) adapterFactory).setFiles(files);
}
-
- switch (adapterFactory.getAdapterType()) {
- case GENERIC:
- ((IGenericAdapterFactory) adapterFactory).configure(configuration, (ARecordType) itemType);
- break;
- case TYPED:
- ((ITypedAdapterFactory) adapterFactory).configure(configuration);
- break;
- }
- return adapterFactory;
+
+ return adapterFactory;
} catch (Exception e) {
throw new AlgebricksException("Unable to create adapter " + e);
}
@@ -539,78 +640,30 @@
return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(scanner, apc);
}
- @SuppressWarnings("rawtypes")
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedIntakeRuntime(JobSpecification jobSpec,
- IDataSource<AqlSourceId> dataSource) throws AlgebricksException {
-
- FeedDataSource feedDataSource = (FeedDataSource) dataSource;
+ public Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IFeedAdapterFactory> buildFeedIntakeRuntime(
+ JobSpecification jobSpec, PrimaryFeed primaryFeed, FeedPolicyAccessor policyAccessor) throws Exception {
+ Triple<IFeedAdapterFactory, ARecordType, AdapterType> factoryOutput = null;
+ factoryOutput = FeedUtil.getPrimaryFeedFactoryAndOutput(primaryFeed, policyAccessor, mdTxnCtx);
+ IFeedAdapterFactory adapterFactory = factoryOutput.first;
FeedIntakeOperatorDescriptor feedIngestor = null;
- Triple<IAdapterFactory, ARecordType, AdapterType> factoryOutput = null;
- AlgebricksPartitionConstraint constraint = null;
-
- try {
- factoryOutput = FeedUtil.getFeedFactoryAndOutput(feedDataSource.getFeed(), mdTxnCtx);
- IAdapterFactory adapterFactory = factoryOutput.first;
- ARecordType adapterOutputType = factoryOutput.second;
- AdapterType adapterType = factoryOutput.third;
-
- 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);
- if (feedPolicy == null) {
- throw new AlgebricksException("Feed not configured with a policy");
- }
- feedPolicy.getProperties().put(BuiltinFeedPolicies.CONFIG_FEED_POLICY_KEY, feedPolicy.getPolicyName());
- switch (adapterType) {
- case INTERNAL:
- feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, new FeedConnectionId(
- feedDataSource.getDatasourceDataverse(), feedDataSource.getDatasourceName(), feedDataSource
- .getFeedConnectionId().getDatasetName()), adapterFactory, adapterOutputType,
- feedDesc, feedPolicy.getProperties());
- break;
- case EXTERNAL:
- String libraryName = feedDataSource.getFeed().getAdapterName().split("#")[0];
- feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, feedDataSource.getFeedConnectionId(),
- libraryName, adapterFactory.getClass().getName(), feedDataSource.getFeed()
- .getAdapterConfiguration(), adapterOutputType, feedDesc, feedPolicy.getProperties());
- break;
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Cofigured feed intake operator with " + adapterType + " adapter");
- }
- constraint = factoryOutput.first.getPartitionConstraint();
- } catch (Exception e) {
- throw new AlgebricksException(e);
+ switch (factoryOutput.third) {
+ case INTERNAL:
+ feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, primaryFeed, adapterFactory,
+ factoryOutput.second, policyAccessor);
+ break;
+ case EXTERNAL:
+ String libraryName = primaryFeed.getAdaptorName().trim()
+ .split(FeedConstants.NamingConstants.LIBRARY_NAME_SEPARATOR)[0];
+ feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, primaryFeed, libraryName, adapterFactory
+ .getClass().getName(), factoryOutput.second, policyAccessor);
+ break;
}
- return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedIngestor, constraint);
- }
- 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);
+ AlgebricksPartitionConstraint partitionConstraint = adapterFactory.getPartitionConstraint();
+ return new Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IFeedAdapterFactory>(feedIngestor,
+ partitionConstraint, adapterFactory);
}
-
- 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(",");
- for (String loc : ingestLocs) {
- 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,
List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
@@ -934,7 +987,7 @@
}
public AqlDataSource lookupSourceInMetadata(AqlSourceId aqlId) throws AlgebricksException, MetadataException {
- Dataset dataset = findDataset(aqlId.getDataverseName(), aqlId.getDatasetName());
+ Dataset dataset = findDataset(aqlId.getDataverseName(), aqlId.getDatasourceName());
if (dataset == null) {
throw new AlgebricksException("Datasource with id " + aqlId + " was not found.");
}
@@ -942,7 +995,7 @@
IAType itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, aqlId.getDataverseName(), tName).getDatatype();
AqlDataSourceType datasourceType = dataset.getDatasetType().equals(DatasetType.EXTERNAL) ? AqlDataSourceType.EXTERNAL_DATASET
: AqlDataSourceType.INTERNAL_DATASET;
- return new DatasetDataSource(aqlId, aqlId.getDataverseName(), aqlId.getDatasetName(), itemType, datasourceType);
+ return new DatasetDataSource(aqlId, aqlId.getDataverseName(), aqlId.getDatasourceName(), itemType, datasourceType);
}
@Override
@@ -971,7 +1024,7 @@
LogicalVariable payload, List<LogicalVariable> additionalNonKeyFields, JobGenContext context,
JobSpecification spec) throws AlgebricksException {
String dataverseName = dataSource.getId().getDataverseName();
- String datasetName = dataSource.getId().getDatasetName();
+ String datasetName = dataSource.getId().getDatasourceName();
Dataset dataset = findDataset(dataverseName, datasetName);
if (dataset == null) {
@@ -1055,7 +1108,7 @@
RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec, boolean bulkload)
throws AlgebricksException {
- String datasetName = dataSource.getId().getDatasetName();
+ String datasetName = dataSource.getId().getDatasourceName();
Dataset dataset = findDataset(dataSource.getId().getDataverseName(), datasetName);
if (dataset == null) {
throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse "
@@ -1173,7 +1226,7 @@
JobGenContext context, JobSpecification spec, boolean bulkload) throws AlgebricksException {
String indexName = dataSourceIndex.getId();
String dataverseName = dataSourceIndex.getDataSource().getId().getDataverseName();
- String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
+ String datasetName = dataSourceIndex.getDataSource().getId().getDatasourceName();
Dataset dataset = findDataset(dataverseName, datasetName);
if (dataset == null) {
@@ -1234,7 +1287,7 @@
String indexName = dataSourceIndex.getId();
String dataverseName = dataSourceIndex.getDataSource().getId().getDataverseName();
- String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
+ String datasetName = dataSourceIndex.getDataSource().getId().getDatasourceName();
IOperatorSchema inputSchema = new OperatorSchemaImpl();
if (inputSchemas.length > 0) {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlSourceId.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlSourceId.java
index 4cee35d..be9f0e2 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlSourceId.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlSourceId.java
@@ -20,23 +20,23 @@
public class AqlSourceId {
private String dataverseName;
- private String datasetName;
+ private String datasourceName;
- public AqlSourceId(String dataverseName, String datasetName) {
+ public AqlSourceId(String dataverseName, String datasourceName) {
this.dataverseName = dataverseName;
- this.datasetName = datasetName;
+ this.datasourceName = datasourceName;
}
@Override
public String toString() {
- return dataverseName + File.pathSeparator + datasetName;
+ return dataverseName + File.pathSeparator + datasourceName;
}
public String getDataverseName() {
return dataverseName;
}
- public String getDatasetName() {
- return datasetName;
+ public String getDatasourceName() {
+ return datasourceName;
}
}
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
index 059a10c..fd06e99 100644
--- 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
@@ -21,7 +21,7 @@
public DatasetDataSource(AqlSourceId id, String datasourceDataverse, String datasourceName, IAType itemType,
AqlDataSourceType datasourceType) throws AlgebricksException {
- super(id, datasourceDataverse, datasourceName, datasourceType);
+ super(id, datasourceDataverse, datasourceName, itemType, datasourceType);
MetadataTransactionContext ctx = null;
try {
ctx = MetadataManager.INSTANCE.beginTransaction();
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
index 695ae31..44402fc 100644
--- 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
@@ -14,32 +14,42 @@
*/
package edu.uci.ics.asterix.metadata.declared;
-import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleListener.ConnectionLocation;
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.entities.Feed.FeedType;
import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.AsterixClusterProperties;
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;
+ private final FeedId sourceFeedId;
+ private final FeedType sourceFeedType;
+ private final ConnectionLocation location;
+ private final String targetDataset;
+ private final String[] locations;
+ private final int computeCardinality;
- public FeedDataSource(AqlSourceId id, FeedConnectionId feedId, IAType itemType, AqlDataSourceType dataSourceType)
+ public FeedDataSource(AqlSourceId id, String targetDataset, IAType itemType, AqlDataSourceType dataSourceType,
+ FeedId sourceFeedId, FeedType sourceFeedType, ConnectionLocation location, String[] locations)
throws AlgebricksException {
- super(id, feedId.getDataverse(), feedId.getFeedName(), dataSourceType);
- this.feedConnectionId = feedId;
- feed = null;
+ super(id, id.getDataverseName(), id.getDatasourceName(), itemType, dataSourceType);
+ this.targetDataset = targetDataset;
+ this.sourceFeedId = sourceFeedId;
+ this.sourceFeedType = sourceFeedType;
+ this.location = location;
+ this.locations = locations;
+ this.computeCardinality = AsterixClusterProperties.INSTANCE.getParticipantNodes().size();
MetadataTransactionContext ctx = null;
try {
MetadataManager.INSTANCE.acquireReadLatch();
ctx = MetadataManager.INSTANCE.beginTransaction();
- feed = MetadataManager.INSTANCE.getFeed(ctx, feedId.getDataverse(), feedId.getFeedName());
- if (feed == null) {
- throw new AlgebricksException("Unknown feed " + feedId);
- }
+ this.feed = MetadataManager.INSTANCE.getFeed(ctx, id.getDataverseName(), id.getDatasourceName());
MetadataManager.INSTANCE.commitTransaction(ctx);
initFeedDataSource(itemType);
} catch (Exception e) {
@@ -71,8 +81,20 @@
return domain;
}
- public FeedConnectionId getFeedConnectionId() {
- return feedConnectionId;
+ public String getTargetDataset() {
+ return targetDataset;
+ }
+
+ public FeedId getSourceFeedId() {
+ return sourceFeedId;
+ }
+
+ public ConnectionLocation getLocation() {
+ return location;
+ }
+
+ public String[] getLocations() {
+ return locations;
}
private void initFeedDataSource(IAType itemType) {
@@ -91,4 +113,12 @@
};
domain = domainForExternalData;
}
+
+ public FeedType getSourceFeedType() {
+ return sourceFeedType;
+ }
+
+ public int getComputeCardinality() {
+ return computeCardinality;
+ }
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapter.java
index 5701292..f4be491 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapter.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapter.java
@@ -4,8 +4,8 @@
import java.nio.ByteBuffer;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.asterix.om.types.ATypeTag;
import edu.uci.ics.asterix.om.types.IAType;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapterFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapterFactory.java
index cbf771b..fd4b6a7 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapterFactory.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/FieldExtractingAdapterFactory.java
@@ -1,7 +1,10 @@
package edu.uci.ics.asterix.metadata.declared;
-import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory.SupportedOperation;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -40,11 +43,7 @@
return "FieldExtractingAdapter[ " + wrappedAdapterFactory.getName() + " ]";
}
- @Override
- public AdapterType getAdapterType() {
- return wrappedAdapterFactory.getAdapterType();
- }
-
+
@Override
public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
return wrappedAdapterFactory.getPartitionConstraint();
@@ -55,5 +54,15 @@
IDatasourceAdapter wrappedAdapter = wrappedAdapterFactory.createAdapter(ctx, partition);
return new FieldExtractingAdapter(ctx, inRecDesc, outRecDesc, extractFields, rType, wrappedAdapter);
}
+
+ @Override
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ wrappedAdapterFactory.configure(configuration, outputType);
+ }
+
+ @Override
+ public ARecordType getAdapterOutputType() {
+ return wrappedAdapterFactory.getAdapterOutputType();
+ }
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/LoadableDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/LoadableDataSource.java
index 2dc840e..c91e3f6 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/LoadableDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/LoadableDataSource.java
@@ -51,7 +51,7 @@
public LoadableDataSource(Dataset targetDataset, IAType itemType, String adapter, Map<String, String> properties)
throws AlgebricksException, IOException {
- super(new AqlSourceId("loadable_dv", "loadable_ds"), "loadable_dv", "loadable_source",
+ super(new AqlSourceId("loadable_dv", "loadable_ds"), "loadable_dv", "loadable_source", itemType,
AqlDataSourceType.LOADABLE);
this.targetDataset = targetDataset;
this.adapter = adapter;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/PKGeneratingAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/PKGeneratingAdapter.java
index 0938ccc..862092a 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/PKGeneratingAdapter.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/PKGeneratingAdapter.java
@@ -18,7 +18,7 @@
import java.util.List;
import edu.uci.ics.asterix.builders.RecordBuilder;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
import edu.uci.ics.asterix.om.base.AMutableUUID;
import edu.uci.ics.asterix.om.base.AUUID;
import edu.uci.ics.asterix.om.pointables.ARecordPointable;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/PKGeneratingAdapterFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/PKGeneratingAdapterFactory.java
index e371b2b..0a36ea8 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/PKGeneratingAdapterFactory.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/PKGeneratingAdapterFactory.java
@@ -14,8 +14,10 @@
*/
package edu.uci.ics.asterix.metadata.declared;
-import edu.uci.ics.asterix.metadata.feeds.IAdapterFactory;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -58,11 +60,6 @@
}
@Override
- public AdapterType getAdapterType() {
- return wrappedAdapterFactory.getAdapterType();
- }
-
- @Override
public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
return wrappedAdapterFactory.getPartitionConstraint();
}
@@ -72,4 +69,14 @@
IDatasourceAdapter wrappedAdapter = wrappedAdapterFactory.createAdapter(ctx, partition);
return new PKGeneratingAdapter(ctx, inRecDesc, outRecDesc, inRecType, outRecType, wrappedAdapter, pkIndex);
}
+
+ @Override
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ wrappedAdapterFactory.configure(configuration, outputType);
+ }
+
+ @Override
+ public ARecordType getAdapterOutputType() {
+ return wrappedAdapterFactory.getAdapterOutputType();
+ }
}
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
index 39aa8ab..53b75a6 100644
--- 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
@@ -15,62 +15,67 @@
package edu.uci.ics.asterix.metadata.entities;
-import java.util.Map;
-
+import edu.uci.ics.asterix.common.feeds.FeedId;
import edu.uci.ics.asterix.common.functions.FunctionSignature;
import edu.uci.ics.asterix.metadata.MetadataCache;
import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
/**
- * Metadata describing a feed.
+ * Feed POJO
*/
public class Feed implements IMetadataEntity {
private static final long serialVersionUID = 1L;
- private final String dataverseName;
- private final String feedName;
- private final String adapterName;
- private final Map<String, String> adapterConfiguration;
- private final FunctionSignature appliedFunction;
+ /** A unique identifier for the feed */
+ protected final FeedId feedId;
- public Feed(String dataverseName, String datasetName, String adapterName, Map<String, String> adapterConfiguration,
- FunctionSignature appliedFunction) {
- this.dataverseName = dataverseName;
- this.feedName = datasetName;
- this.adapterName = adapterName;
- this.adapterConfiguration = adapterConfiguration;
+ /** The function that is to be applied on each incoming feed tuple **/
+ protected final FunctionSignature appliedFunction;
+
+ /** The type {@code FeedType} associated with the feed. **/
+ protected final FeedType feedType;
+
+ /** A string representation of the instance **/
+ protected final String displayName;
+
+ public enum FeedType {
+ /**
+ * A feed that derives its data from an external source.
+ */
+ PRIMARY,
+
+ /**
+ * A feed that derives its data from another primary or secondary feed.
+ */
+ SECONDARY
+ }
+
+ public Feed(String dataverseName, String datasetName, FunctionSignature appliedFunction, FeedType feedType) {
+ this.feedId = new FeedId(dataverseName, datasetName);
this.appliedFunction = appliedFunction;
+ this.feedType = feedType;
+ this.displayName = feedType + "(" + feedId + ")";
+ }
+
+ public FeedId getFeedId() {
+ return feedId;
}
public String getDataverseName() {
- return dataverseName;
+ return feedId.getDataverse();
}
public String getFeedName() {
- return feedName;
- }
-
- public String getAdapterName() {
- return adapterName;
- }
-
- public Map<String, String> getAdapterConfiguration() {
- return adapterConfiguration;
+ return feedId.getFeedName();
}
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);
+ public FeedType getFeedType() {
+ return feedType;
}
@Override
@@ -81,13 +86,27 @@
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;
+ Feed otherFeed = (Feed) other;
+ return otherFeed.getFeedId().equals(feedId);
+ }
+
+ @Override
+ public int hashCode() {
+ return displayName.hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return feedType + "(" + feedId + ")";
+ }
+
+ @Override
+ public Object addToCache(MetadataCache cache) {
+ return cache.addFeedIfNotExists(this);
+ }
+
+ @Override
+ public Object dropFromCache(MetadataCache cache) {
+ return cache.dropFeed(this);
}
}
\ 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
deleted file mode 100644
index 679276f..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedActivity.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/*
- * 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/PrimaryFeed.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/PrimaryFeed.java
new file mode 100644
index 0000000..62f0d07
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/PrimaryFeed.java
@@ -0,0 +1,76 @@
+/*
+ * 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 java.util.Map.Entry;
+
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+
+/**
+ * A primary feed is one that derives its data from an external source via an adaptor.
+ * This class is a holder object for the metadata associated with a primary feed.
+ */
+public class PrimaryFeed extends Feed implements IMetadataEntity {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String adaptorName;
+ private final Map<String, String> adaptorConfiguration;
+
+ public PrimaryFeed(String dataverseName, String datasetName, String adaptorName,
+ Map<String, String> adaptorConfiguration, FunctionSignature appliedFunction) {
+ super(dataverseName, datasetName, appliedFunction, FeedType.PRIMARY);
+ this.adaptorName = adaptorName;
+ this.adaptorConfiguration = adaptorConfiguration;
+ }
+
+ public String getAdaptorName() {
+ return adaptorName;
+ }
+
+ public Map<String, String> getAdaptorConfiguration() {
+ return adaptorConfiguration;
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (!super.equals(other) || !(other instanceof PrimaryFeed)) {
+ return false;
+ }
+
+ PrimaryFeed otherFeed = (PrimaryFeed) other;
+ if (!otherFeed.getAdaptorName().equals(adaptorName)) {
+ return false;
+ }
+
+ for (Entry<String, String> entry : adaptorConfiguration.entrySet()) {
+ if (!(entry.getValue().equals(otherFeed.getAdaptorConfiguration().get(entry.getKey())))) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return "PrimaryFeed (" + adaptorName + ")";
+ }
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/SecondaryFeed.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/SecondaryFeed.java
new file mode 100644
index 0000000..c1f51ba
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/SecondaryFeed.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.metadata.entities;
+
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+
+/**
+ * A secondary feed is one that derives its data from another (primary/secondary) feed.
+ * This class is a holder object for the metadata associated with a secondary feed.
+ */
+public class SecondaryFeed extends Feed implements IMetadataEntity {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String sourceFeedName;
+
+ public SecondaryFeed(String dataverseName, String feedName, String sourceFeedName, FunctionSignature appliedFunction) {
+ super(dataverseName, feedName, appliedFunction, FeedType.SECONDARY);
+ this.sourceFeedName = sourceFeedName;
+ }
+
+ public String getSourceFeedName() {
+ return sourceFeedName;
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (this == other) {
+ return true;
+ }
+ if (!super.equals(other) || !(other instanceof SecondaryFeed)) {
+ return false;
+ }
+
+ SecondaryFeed otherFeed = (SecondaryFeed) other;
+ if (!otherFeed.getSourceFeedName().equals(sourceFeedName)) {
+ return false;
+ }
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return "SecondaryFeed (" + feedId + ")" + "<--" + "(" + sourceFeedName + ")";
+ }
+}
\ No newline at end of file
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 1bb34d2..580881d 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
@@ -85,7 +85,7 @@
aString.setValue(adapter.getAdapterIdentifier().getNamespace());
stringSerde.serialize(aString, tupleBuilder.getDataOutput());
tupleBuilder.addFieldEndOffset();
- aString.setValue(adapter.getAdapterIdentifier().getAdapterName());
+ aString.setValue(adapter.getAdapterIdentifier().getName());
stringSerde.serialize(aString, tupleBuilder.getDataOutput());
tupleBuilder.addFieldEndOffset();
@@ -101,7 +101,7 @@
// write field 1
fieldValue.reset();
- aString.setValue(adapter.getAdapterIdentifier().getAdapterName());
+ aString.setValue(adapter.getAdapterIdentifier().getName());
stringSerde.serialize(aString, fieldValue.getDataOutput());
recordBuilder.addField(MetadataRecordTypes.DATASOURCE_ADAPTER_ARECORD_NAME_FIELD_INDEX, fieldValue);
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
deleted file mode 100644
index 6c71036..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedActivityTupleTranslator.java
+++ /dev/null
@@ -1,244 +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.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 getMetadataEntityFromTuple(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/FeedTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/FeedTupleTranslator.java
index def0938..32ed123 100644
--- 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
@@ -25,8 +25,8 @@
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.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;
@@ -34,8 +34,9 @@
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.metadata.entities.Feed.FeedType;
+import edu.uci.ics.asterix.metadata.entities.PrimaryFeed;
+import edu.uci.ics.asterix.metadata.entities.SecondaryFeed;
import edu.uci.ics.asterix.om.base.AMutableString;
import edu.uci.ics.asterix.om.base.ANull;
import edu.uci.ics.asterix.om.base.ARecord;
@@ -65,14 +66,9 @@
@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
@@ -92,49 +88,64 @@
.getValueByPos(MetadataRecordTypes.FEED_ARECORD_DATAVERSE_NAME_FIELD_INDEX)).getStringValue();
String feedName = ((AString) feedRecord.getValueByPos(MetadataRecordTypes.FEED_ARECORD_FEED_NAME_FIELD_INDEX))
.getStringValue();
- String adapterName = ((AString) feedRecord
- .getValueByPos(MetadataRecordTypes.FEED_ARECORD_ADAPTER_NAME_FIELD_INDEX)).getStringValue();
-
- IACursor cursor = ((AUnorderedList) feedRecord
- .getValueByPos(MetadataRecordTypes.FEED_ARECORD_ADAPTER_CONFIGURATION_FIELD_INDEX)).getCursor();
- String key;
- String value;
- Map<String, String> adapterConfiguration = 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();
- adapterConfiguration.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]));
+ String functionName = ((AString) o).getStringValue();
+ signature = new FunctionSignature(dataverseName, functionName, 1);
}
- feed = new Feed(dataverseName, feedName, adapterName, adapterConfiguration, signature);
+ String feedType = ((AString) feedRecord.getValueByPos(MetadataRecordTypes.FEED_ARECORD_FEED_TYPE_FIELD_INDEX))
+ .getStringValue();
+
+ FeedType feedTypeEnum = FeedType.valueOf(feedType.toUpperCase());
+ switch (feedTypeEnum) {
+ case PRIMARY: {
+ ARecord feedTypeDetailsRecord = (ARecord) feedRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ARECORD_PRIMARY_TYPE_DETAILS_FIELD_INDEX);
+ String adapterName = ((AString) feedTypeDetailsRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_NAME_FIELD_INDEX))
+ .getStringValue();
+
+ IACursor cursor = ((AUnorderedList) feedTypeDetailsRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ARECORD_PRIMARY_FIELD_DETAILS_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);
+ }
+ feed = new PrimaryFeed(dataverseName, feedName, adapterName, adaptorConfiguration, signature);
+
+ }
+ break;
+ case SECONDARY: {
+ ARecord feedTypeDetailsRecord = (ARecord) feedRecord
+ .getValueByPos(MetadataRecordTypes.FEED_ARECORD_SECONDARY_TYPE_DETAILS_FIELD_INDEX);
+
+ String sourceFeedName = ((AString) feedTypeDetailsRecord
+ .getValueByPos(MetadataRecordTypes.FEED_TYPE_SECONDARY_ARECORD_SOURCE_FEED_NAME_FIELD_INDEX))
+ .getStringValue();
+
+ feed = new SecondaryFeed(dataverseName, feedName, sourceFeedName, signature);
+
+ }
+ break;
+ }
+
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();
-
+ // write the key in the first two fields of the tuple
tupleBuilder.reset();
aString.setValue(feed.getDataverseName());
stringSerde.serialize(aString, tupleBuilder.getDataOutput());
@@ -160,35 +171,23 @@
// write field 2
fieldValue.reset();
- aString.setValue(feed.getAdapterName());
- stringSerde.serialize(aString, fieldValue.getDataOutput());
- recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_ADAPTER_NAME_FIELD_INDEX, fieldValue);
-
- // write field 3 (adapterConfiguration)
- Map<String, String> adapterConfiguration = feed.getAdapterConfiguration();
- UnorderedListBuilder listBuilder = new UnorderedListBuilder();
- listBuilder
- .reset((AUnorderedListType) MetadataRecordTypes.FEED_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.FEED_ARECORD_ADAPTER_CONFIGURATION_FIELD_INDEX]);
- for (Map.Entry<String, String> property : adapterConfiguration.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_ADAPTER_CONFIGURATION_FIELD_INDEX, fieldValue);
-
- // write field 4
- fieldValue.reset();
if (feed.getAppliedFunction() != null) {
- aString.setValue(feed.getAppliedFunction().toString());
+ aString.setValue(feed.getAppliedFunction().getName());
stringSerde.serialize(aString, fieldValue.getDataOutput());
recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_FUNCTION_FIELD_INDEX, fieldValue);
}
- // write field 5
+ // write field 3
+ fieldValue.reset();
+ aString.setValue(feed.getFeedType().name().toUpperCase());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_FEED_TYPE_FIELD_INDEX, fieldValue);
+
+ // write field 4/5
+ fieldValue.reset();
+ writeFeedTypeDetailsRecordType(recordBuilder, feed, fieldValue);
+
+ // write field 6
fieldValue.reset();
aString.setValue(Calendar.getInstance().getTime().toString());
stringSerde.serialize(aString, fieldValue.getDataOutput());
@@ -206,6 +205,85 @@
return tuple;
}
+ @SuppressWarnings("unchecked")
+ private void writeFeedTypeDetailsRecordType(IARecordBuilder recordBuilder, Feed feed,
+ ArrayBackedValueStorage fieldValue) throws HyracksDataException {
+
+ switch (feed.getFeedType()) {
+ case PRIMARY: {
+ PrimaryFeed primaryFeed = (PrimaryFeed) feed;
+
+ IARecordBuilder primaryDetailsRecordBuilder = new RecordBuilder();
+ OrderedListBuilder listBuilder = new OrderedListBuilder();
+ ArrayBackedValueStorage primaryRecordfieldValue = new ArrayBackedValueStorage();
+ ArrayBackedValueStorage primaryRecordItemValue = new ArrayBackedValueStorage();
+ primaryDetailsRecordBuilder.reset(MetadataRecordTypes.PRIMARY_FEED_DETAILS_RECORDTYPE);
+
+ AMutableString aString = new AMutableString("");
+ ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ASTRING);
+
+ // write field 0
+ fieldValue.reset();
+ aString.setValue(primaryFeed.getAdaptorName());
+ stringSerde.serialize(aString, primaryRecordfieldValue.getDataOutput());
+ primaryDetailsRecordBuilder.addField(
+ MetadataRecordTypes.FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_NAME_FIELD_INDEX,
+ primaryRecordfieldValue);
+
+ // write field 1
+ listBuilder
+ .reset((AUnorderedListType) MetadataRecordTypes.PRIMARY_FEED_DETAILS_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_CONFIGURATION_FIELD_INDEX]);
+ for (Map.Entry<String, String> property : primaryFeed.getAdaptorConfiguration().entrySet()) {
+ String name = property.getKey();
+ String value = property.getValue();
+ primaryRecordItemValue.reset();
+ writePropertyTypeRecord(name, value, primaryRecordItemValue.getDataOutput());
+ listBuilder.addItem(primaryRecordItemValue);
+ }
+ primaryRecordfieldValue.reset();
+ listBuilder.write(primaryRecordfieldValue.getDataOutput(), true);
+ primaryDetailsRecordBuilder.addField(
+ MetadataRecordTypes.FEED_ARECORD_PRIMARY_FIELD_DETAILS_ADAPTOR_CONFIGURATION_FIELD_INDEX,
+ primaryRecordfieldValue);
+
+ try {
+ primaryDetailsRecordBuilder.write(fieldValue.getDataOutput(), true);
+ } catch (IOException | AsterixException e) {
+ throw new HyracksDataException(e);
+ }
+
+ recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_PRIMARY_TYPE_DETAILS_FIELD_INDEX, fieldValue);
+ }
+ break;
+
+ case SECONDARY:
+ SecondaryFeed secondaryFeed = (SecondaryFeed) feed;
+
+ IARecordBuilder secondaryDetailsRecordBuilder = new RecordBuilder();
+ ArrayBackedValueStorage secondaryFieldValue = new ArrayBackedValueStorage();
+ secondaryDetailsRecordBuilder.reset(MetadataRecordTypes.SECONDARY_FEED_DETAILS_RECORDTYPE);
+
+ // write field 0
+ fieldValue.reset();
+ aString.setValue(secondaryFeed.getSourceFeedName());
+ stringSerde.serialize(aString, secondaryFieldValue.getDataOutput());
+ secondaryDetailsRecordBuilder.addField(
+ MetadataRecordTypes.FEED_ARECORD_SECONDARY_FIELD_DETAILS_SOURCE_FEED_NAME_FIELD_INDEX,
+ secondaryFieldValue);
+
+ try {
+ secondaryDetailsRecordBuilder.write(fieldValue.getDataOutput(), true);
+ } catch (IOException | AsterixException e) {
+ throw new HyracksDataException(e);
+ }
+ recordBuilder.addField(MetadataRecordTypes.FEED_ARECORD_SECONDARY_TYPE_DETAILS_FIELD_INDEX, fieldValue);
+ break;
+ }
+
+ }
+
+ @SuppressWarnings("unchecked")
public void writePropertyTypeRecord(String name, String value, DataOutput out) throws HyracksDataException {
IARecordBuilder propertyRecordBuilder = new RecordBuilder();
ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
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/external/IAdapterFactory.java
similarity index 81%
rename from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IAdapterFactory.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/external/IAdapterFactory.java
index f0a3aa4..7fc2e7e 100644
--- 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/external/IAdapterFactory.java
@@ -12,10 +12,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.metadata.feeds;
+package edu.uci.ics.asterix.metadata.external;
import java.io.Serializable;
+import java.util.Map;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -26,14 +29,7 @@
*/
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 static final String KEY_TYPE_NAME = "type-name";
public enum SupportedOperation {
READ,
@@ -59,14 +55,7 @@
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
+ * Gets 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
@@ -88,4 +77,18 @@
*/
public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception;
+ /**
+ * @param configuration
+ * @param outputType
+ * @throws Exception
+ */
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception;
+
+ /**
+ * Gets the record type associated with the output of the adapter
+ *
+ * @return
+ */
+ public ARecordType getAdapterOutputType();
+
}
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
index 9e8e5f7..3bc402b 100644
--- 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
@@ -14,9 +14,9 @@
*/
package edu.uci.ics.asterix.metadata.feeds;
-import java.util.HashMap;
import java.util.Map;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
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;
@@ -29,23 +29,11 @@
private static final long serialVersionUID = 1L;
+ public static final String KEY_PARSER_FACTORY = "parser";
+
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..86ca550
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AbstractFeedDatasourceAdapter.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+
+
+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-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterExecutor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterExecutor.java
new file mode 100644
index 0000000..6bd726c
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterExecutor.java
@@ -0,0 +1,56 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.feeds.DistributeFeedFrameWriter;
+import edu.uci.ics.asterix.common.feeds.api.IAdapterRuntimeManager;
+import edu.uci.ics.asterix.common.feeds.api.IAdapterRuntimeManager.State;
+import edu.uci.ics.asterix.common.feeds.api.IFeedAdapter;
+
+public class AdapterExecutor implements Runnable {
+
+ private static final Logger LOGGER = Logger.getLogger(AdapterExecutor.class.getName());
+
+ private final DistributeFeedFrameWriter writer;
+
+ private final IFeedAdapter adapter;
+
+ private final IAdapterRuntimeManager adapterManager;
+
+ public AdapterExecutor(int partition, DistributeFeedFrameWriter writer, IFeedAdapter adapter,
+ IAdapterRuntimeManager adapterManager) {
+ this.writer = writer;
+ this.adapter = adapter;
+ this.adapterManager = adapterManager;
+ }
+
+ @Override
+ public void run() {
+ int partition = adapterManager.getPartition();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Starting ingestion for partition:" + partition);
+ }
+ boolean continueIngestion = true;
+ boolean failedIngestion = false;
+ while (continueIngestion) {
+ try {
+ adapter.start(partition, writer);
+ continueIngestion = false;
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Exception during feed ingestion " + e.getMessage());
+ e.printStackTrace();
+ }
+ continueIngestion = adapter.handleException(e);
+ failedIngestion = !continueIngestion;
+ }
+ }
+
+ adapterManager.setState(failedIngestion ? State.FAILED_INGESTION : State.FINISHED_INGESTION);
+ synchronized (adapterManager) {
+ adapterManager.notifyAll();
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterIdentifier.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterIdentifier.java
index 897faae..b032bab 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterIdentifier.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/AdapterIdentifier.java
@@ -17,40 +17,46 @@
import java.io.Serializable;
/**
- * A unique identifier for a datasource adapter.
+ * A unique identifier for a data source adapter.
*/
public class AdapterIdentifier implements Serializable {
private static final long serialVersionUID = 1L;
private final String namespace;
- private final String adapterName;
+ private final String name;
- public AdapterIdentifier(String namespace, String adapterName) {
+ public AdapterIdentifier(String namespace, String name) {
this.namespace = namespace;
- this.adapterName = adapterName;
+ this.name = name;
}
public String getNamespace() {
return namespace;
}
- public String getAdapterName() {
- return adapterName;
+ public String getName() {
+ return name;
}
@Override
public int hashCode() {
- return (namespace + "@" + adapterName).hashCode();
+ return (namespace + "@" + name).hashCode();
}
@Override
public boolean equals(Object o) {
+ if (o == null) {
+ return false;
+ }
+ if (this == o) {
+ return true;
+ }
if (!(o instanceof AdapterIdentifier)) {
return false;
}
return namespace.equals(((AdapterIdentifier) o).getNamespace())
- && namespace.equals(((AdapterIdentifier) o).getNamespace());
+ && name.equals(((AdapterIdentifier) o).getName());
}
}
\ No newline at end of file
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
index b9a5e73..de288ba 100644
--- 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
@@ -15,68 +15,51 @@
package edu.uci.ics.asterix.metadata.feeds;
import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.Executors;
import java.util.logging.Level;
import java.util.logging.Logger;
-import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
-import edu.uci.ics.asterix.common.feeds.IFeedManager;
-import edu.uci.ics.asterix.metadata.feeds.FeedFrameWriter.Mode;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.asterix.common.feeds.DistributeFeedFrameWriter;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.IngestionRuntime;
+import edu.uci.ics.asterix.common.feeds.api.IAdapterRuntimeManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
-public class AdapterRuntimeManager implements IAdapterExecutor {
+public class AdapterRuntimeManager implements IAdapterRuntimeManager {
private static final Logger LOGGER = Logger.getLogger(AdapterRuntimeManager.class.getName());
- private final FeedConnectionId feedId;
+ private final FeedId feedId;
- private IFeedAdapter feedAdapter;
+ private final IFeedAdapter feedAdapter;
- private AdapterExecutor adapterExecutor;
+ private final IIntakeProgressTracker tracker;
- private State state;
+ private final AdapterExecutor adapterExecutor;
- private int partition;
+ private final int partition;
+
+ private final ExecutorService executorService;
private IngestionRuntime ingestionRuntime;
- private final IFeedManager feedManager;
+ private State state;
- 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, IFeedManager feedManager) {
+ public AdapterRuntimeManager(FeedId feedId, IFeedAdapter feedAdapter, IIntakeProgressTracker tracker,
+ DistributeFeedFrameWriter writer, int partition) {
this.feedId = feedId;
this.feedAdapter = feedAdapter;
+ this.tracker = tracker;
this.partition = partition;
- this.feedManager = feedManager;
this.adapterExecutor = new AdapterExecutor(partition, writer, feedAdapter, this);
+ this.executorService = Executors.newSingleThreadExecutor();
+ this.state = State.INACTIVE_INGESTION;
}
@Override
public void start() throws Exception {
state = State.ACTIVE_INGESTION;
- ingestionRuntime = new IngestionRuntime(feedId, partition, FeedRuntimeType.INGESTION, this);
- feedManager.registerFeedRuntime(ingestionRuntime);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Registered feed runtime manager for " + this.getFeedId());
- }
- ExecutorService executorService = feedManager.getFeedExecutorService(feedId);
executorService.execute(adapterExecutor);
}
@@ -84,19 +67,18 @@
public void stop() {
try {
feedAdapter.stop();
- state = State.FINISHED_INGESTION;
- synchronized (this) {
- notifyAll();
- }
} catch (Exception exception) {
if (LOGGER.isLoggable(Level.SEVERE)) {
LOGGER.severe("Unable to stop adapter " + feedAdapter + ", encountered exception " + exception);
}
+ } finally {
+ state = State.FINISHED_INGESTION;
+ executorService.shutdown();
}
}
@Override
- public FeedConnectionId getFeedId() {
+ public FeedId getFeedId() {
return feedId;
}
@@ -109,84 +91,15 @@
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) {
- e.printStackTrace();
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe("Exception during feed ingestion " + e.getMessage());
- }
- } 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 IIntakeProgressTracker getTracker() {
+ return tracker;
}
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;
- }
+ public synchronized void setState(State state) {
this.state = state;
}
@@ -194,6 +107,7 @@
return adapterExecutor;
}
+ @Override
public int getPartition() {
return partition;
}
@@ -202,4 +116,9 @@
return ingestionRuntime;
}
+ @Override
+ public IIntakeProgressTracker getProgressTracker() {
+ return tracker;
+ }
+
}
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
index 3bd73a3..3d29aeb 100644
--- 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
@@ -17,6 +17,7 @@
import java.util.HashMap;
import java.util.Map;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
@@ -26,16 +27,22 @@
public static final FeedPolicy BASIC = initializeBasicPolicy();
- public static final FeedPolicy BASIC_MONITORED = initializeBasicMonitoredPolicy();
+ public static final FeedPolicy BASIC_FT = initializeBasicFTPolicy();
- public static final FeedPolicy FAULT_TOLERANT_BASIC_MONITORED = initializeFaultTolerantBasicMonitoredPolicy();
+ public static final FeedPolicy ADVANCED_FT = initializeAdvancedFTPolicy();
- public static final FeedPolicy ELASTIC = initializeFaultTolerantBasicMonitoredElasticPolicy();
+ public static final FeedPolicy ADVANCED_FT_DISCARD = initializeAdvancedFTDiscardPolicy();
- public static final FeedPolicy[] policies = new FeedPolicy[] { BRITTLE, BASIC, BASIC_MONITORED,
- FAULT_TOLERANT_BASIC_MONITORED, ELASTIC };
+ public static final FeedPolicy ADVANCED_FT_SPILL = initializeAdvancedFTSpillPolicy();
- public static final FeedPolicy DEFAULT_POLICY = BASIC;
+ public static final FeedPolicy ADVANCED_FT_THROTTLE = initializeAdvancedFTThrottlePolicy();
+
+ public static final FeedPolicy ELASTIC = initializeAdvancedFTElasticPolicy();
+
+ public static final FeedPolicy[] policies = new FeedPolicy[] { BRITTLE, BASIC, BASIC_FT, ADVANCED_FT,
+ ADVANCED_FT_DISCARD, ADVANCED_FT_SPILL, ADVANCED_FT_THROTTLE, ELASTIC };
+
+ public static final FeedPolicy DEFAULT_POLICY = BASIC_FT;
public static final String CONFIG_FEED_POLICY_KEY = "policy";
@@ -48,79 +55,134 @@
return null;
}
- // BMFE
- 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);
- }
-
- //BMF
- 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);
- }
-
- //BM
- 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);
- }
-
- //B
- 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);
- }
-
- //Br
+ //Brittle
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.SOFT_FAILURE_CONTINUE, "false");
+ policyParams.put(FeedPolicyAccessor.SOFT_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");
+ policyParams.put(FeedPolicyAccessor.TIME_TRACKING, "false");
+ policyParams.put(FeedPolicyAccessor.AT_LEAST_ONE_SEMANTICS, "false");
+
String description = "Brittle";
- return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "Br", description, policyParams);
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "Brittle", description, policyParams);
+ }
+
+ //Basic
+ private static FeedPolicy initializeBasicPolicy() {
+ Map<String, String> policyParams = new HashMap<String, String>();
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_CONTINUE, "false");
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_LOG_DATA, "true");
+ policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+ policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+ policyParams.put(FeedPolicyAccessor.TIME_TRACKING, "false");
+ policyParams.put(FeedPolicyAccessor.AT_LEAST_ONE_SEMANTICS, "false");
+
+ String description = "Basic";
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "Basic", description, policyParams);
+ }
+
+ // BasicFT
+ private static FeedPolicy initializeBasicFTPolicy() {
+ Map<String, String> policyParams = new HashMap<String, String>();
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_LOG_DATA, "true");
+ policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "false");
+ policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+ policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+ policyParams.put(FeedPolicyAccessor.SPILL_TO_DISK_ON_CONGESTION, "false");
+ policyParams.put(FeedPolicyAccessor.MAX_FRACTION_DISCARD, "1");
+ policyParams.put(FeedPolicyAccessor.TIME_TRACKING, "false");
+ policyParams.put(FeedPolicyAccessor.AT_LEAST_ONE_SEMANTICS, "false");
+ policyParams.put(FeedPolicyAccessor.THROTTLING_ENABLED, "false");
+
+ String description = "Basic Monitored Fault-Tolerant";
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "BasicFT", description, policyParams);
+ }
+
+ // AdvancedFT
+ private static FeedPolicy initializeAdvancedFTPolicy() {
+ Map<String, String> policyParams = new HashMap<String, String>();
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_LOG_DATA, "true");
+ policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+ policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+ policyParams.put(FeedPolicyAccessor.TIME_TRACKING, "true");
+ policyParams.put(FeedPolicyAccessor.AT_LEAST_ONE_SEMANTICS, "true");
+
+ String description = "Basic Monitored Fault-Tolerant with at least once semantics";
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "AdvancedFT", description, policyParams);
+ }
+
+ // AdvancedFT_Discard
+ private static FeedPolicy initializeAdvancedFTDiscardPolicy() {
+ Map<String, String> policyParams = new HashMap<String, String>();
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_LOG_DATA, "true");
+ policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+ policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+ policyParams.put(FeedPolicyAccessor.MAX_SPILL_SIZE_ON_DISK, "false");
+ policyParams.put(FeedPolicyAccessor.MAX_FRACTION_DISCARD, "100");
+ policyParams.put(FeedPolicyAccessor.TIME_TRACKING, "false");
+ policyParams.put(FeedPolicyAccessor.LOGGING_STATISTICS, "true");
+
+ String description = "AdvancedFT 100% Discard during congestion";
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "AdvancedFT_Discard", description,
+ policyParams);
+ }
+
+ // AdvancedFT_Spill
+ private static FeedPolicy initializeAdvancedFTSpillPolicy() {
+ Map<String, String> policyParams = new HashMap<String, String>();
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_LOG_DATA, "true");
+ policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+ policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+ policyParams.put(FeedPolicyAccessor.SPILL_TO_DISK_ON_CONGESTION, "" + Boolean.TRUE);
+ policyParams.put(FeedPolicyAccessor.MAX_SPILL_SIZE_ON_DISK, "" + FeedPolicyAccessor.NO_LIMIT);
+ policyParams.put(FeedPolicyAccessor.TIME_TRACKING, "true");
+
+ String description = "AdvancedFT 100% Discard during congestion";
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "AdvancedFT_Spill", description, policyParams);
+ }
+
+ // AdvancedFT_Spill
+ private static FeedPolicy initializeAdvancedFTThrottlePolicy() {
+ Map<String, String> policyParams = new HashMap<String, String>();
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_LOG_DATA, "true");
+ policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+ policyParams.put(FeedPolicyAccessor.ELASTIC, "false");
+ policyParams.put(FeedPolicyAccessor.SPILL_TO_DISK_ON_CONGESTION, "" + Boolean.FALSE);
+ policyParams.put(FeedPolicyAccessor.MAX_FRACTION_DISCARD, "" + 0);
+ policyParams.put(FeedPolicyAccessor.TIME_TRACKING, "false");
+ policyParams.put(FeedPolicyAccessor.THROTTLING_ENABLED, "true");
+
+ String description = "AdvancedFT Throttle during congestion";
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "AdvancedFT_Throttle", description,
+ policyParams);
+ }
+
+ // AdvancedFT_Elastic
+ private static FeedPolicy initializeAdvancedFTElasticPolicy() {
+ Map<String, String> policyParams = new HashMap<String, String>();
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.SOFT_FAILURE_LOG_DATA, "true");
+ policyParams.put(FeedPolicyAccessor.HARDWARE_FAILURE_CONTINUE, "true");
+ policyParams.put(FeedPolicyAccessor.CLUSTER_REBOOT_AUTO_RESTART, "true");
+ policyParams.put(FeedPolicyAccessor.ELASTIC, "true");
+ policyParams.put(FeedPolicyAccessor.TIME_TRACKING, "false");
+ policyParams.put(FeedPolicyAccessor.LOGGING_STATISTICS, "true");
+
+ String description = "Basic Monitored Fault-Tolerant Elastic";
+ return new FeedPolicy(MetadataConstants.METADATA_DATAVERSE_NAME, "AdvancedFT_Elastic", description,
+ policyParams);
}
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/CollectTransformFeedFrameWriter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/CollectTransformFeedFrameWriter.java
new file mode 100644
index 0000000..398de35
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/CollectTransformFeedFrameWriter.java
@@ -0,0 +1,102 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedOperatorOutputSideHandler;
+import edu.uci.ics.asterix.common.feeds.api.ISubscribableRuntime;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+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.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+
+public class CollectTransformFeedFrameWriter implements IFeedOperatorOutputSideHandler {
+
+ private final FeedConnectionId connectionId;
+ private IFrameWriter downstreamWriter;
+ private final FrameTupleAccessor inputFrameTupleAccessor;
+ private final FrameTupleAppender tupleAppender;
+ private final IFrame frame;
+
+ private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(1);
+
+ public CollectTransformFeedFrameWriter(IHyracksTaskContext ctx, IFrameWriter downstreamWriter,
+ ISubscribableRuntime sourceRuntime, RecordDescriptor outputRecordDescriptor, FeedConnectionId connectionId)
+ throws HyracksDataException {
+ this.downstreamWriter = downstreamWriter;
+ RecordDescriptor inputRecordDescriptor = sourceRuntime.getRecordDescriptor();
+ inputFrameTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
+ tupleAppender = new FrameTupleAppender();
+ frame = new VSizeFrame(ctx);
+ tupleAppender.reset(frame, true);
+ this.connectionId = connectionId;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ downstreamWriter.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ inputFrameTupleAccessor.reset(buffer);
+ int nTuple = inputFrameTupleAccessor.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+ tupleBuilder.addField(inputFrameTupleAccessor, t, 0);
+ appendTupleToFrame();
+ tupleBuilder.reset();
+ }
+ }
+
+ private void appendTupleToFrame() throws HyracksDataException {
+ if (!tupleAppender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+ tupleBuilder.getSize())) {
+ FrameUtils.flushFrame(frame.getBuffer(), downstreamWriter);
+ tupleAppender.reset(frame, true);
+ if (!tupleAppender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+ tupleBuilder.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ downstreamWriter.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ downstreamWriter.close();
+ }
+
+ @Override
+ public FeedId getFeedId() {
+ return connectionId.getFeedId();
+ }
+
+ @Override
+ public Type getType() {
+ return Type.COLLECT_TRANSFORM_FEED_OUTPUT_HANDLER;
+ }
+
+ public IFrameWriter getDownstreamWriter() {
+ return downstreamWriter;
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public void reset(IFrameWriter writer) {
+ this.downstreamWriter = writer;
+ }
+
+}
\ No newline at end of file
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
deleted file mode 100644
index b71be24..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ConditionalPushTupleParserFactory.java
+++ /dev/null
@@ -1,213 +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.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 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.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;
-
- @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, quote, hasHeader);
- 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 char quote;
- private boolean hasHeader;
- private final ParserType parserType;
-
- public enum ParserType {
- ADM,
- DELIMITED_DATA
- }
-
- public ConditionalPushTupleParserFactory(ARecordType recordType, IValueParserFactory[] valueParserFactories,
- char fieldDelimiter, char quote, boolean hasHeader, Map<String, String> configuration) {
- this.recordType = recordType;
- this.valueParserFactories = valueParserFactories;
- this.delimiter = fieldDelimiter;
- this.quote = quote;
- this.hasHeader = hasHeader;
- 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);
- 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) {
- appender.flush(writer, true);
- }
- } else {
- appender.flush(writer, true);
- }
- }
- } 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())) {
- appender.flush(writer, 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 + ")");
- }
- appender.flush(writer, true);
- tuplesInFrame = 0;
- }
- }
- } catch (HyracksDataException e) {
- e.printStackTrace();
- }
- }
-
- }
-
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
index 06de1e0..536c702 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ExternalDataScanOperatorDescriptor.java
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.asterix.metadata.feeds;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory;
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;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedCollectOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedCollectOperatorDescriptor.java
new file mode 100644
index 0000000..b67b813
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedCollectOperatorDescriptor.java
@@ -0,0 +1,166 @@
+/*
+ * 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.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.IngestionRuntime;
+import edu.uci.ics.asterix.common.feeds.SubscribableFeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleListener.ConnectionLocation;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedSubscriptionManager;
+import edu.uci.ics.asterix.common.feeds.api.ISubscribableRuntime;
+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;
+
+/**
+ * FeedCollectOperatorDescriptor 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 FeedCollectOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ private static final Logger LOGGER = Logger.getLogger(FeedCollectOperatorDescriptor.class.getName());
+
+ /** The type associated with the ADM data output from the feed adaptor */
+ private final IAType outputType;
+
+ /** unique identifier for a feed instance. */
+ private final FeedConnectionId connectionId;
+
+ /** Map representation of policy parameters */
+ private final Map<String, String> feedPolicyProperties;
+
+ /** The (singleton) instance of {@code IFeedIngestionManager} **/
+ private IFeedSubscriptionManager subscriptionManager;
+
+ /** The source feed from which the feed derives its data from. **/
+ private final FeedId sourceFeedId;
+
+ /** The subscription location at which the recipient feed receives tuples from the source feed **/
+ private final ConnectionLocation subscriptionLocation;
+
+ public FeedCollectOperatorDescriptor(JobSpecification spec, FeedConnectionId feedConnectionId, FeedId sourceFeedId,
+ ARecordType atype, RecordDescriptor rDesc, Map<String, String> feedPolicyProperties,
+ ConnectionLocation subscriptionLocation) {
+ super(spec, 0, 1);
+ recordDescriptors[0] = rDesc;
+ this.outputType = atype;
+ this.connectionId = feedConnectionId;
+ this.feedPolicyProperties = feedPolicyProperties;
+ this.sourceFeedId = sourceFeedId;
+ this.subscriptionLocation = subscriptionLocation;
+ }
+
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+ throws HyracksDataException {
+ IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+ .getApplicationContext().getApplicationObject();
+ this.subscriptionManager = runtimeCtx.getFeedManager().getFeedSubscriptionManager();
+ ISubscribableRuntime sourceRuntime = null;
+ IOperatorNodePushable nodePushable = null;
+ switch (subscriptionLocation) {
+ case SOURCE_FEED_INTAKE_STAGE:
+ try {
+ SubscribableFeedRuntimeId feedSubscribableRuntimeId = new SubscribableFeedRuntimeId(sourceFeedId,
+ FeedRuntimeType.INTAKE, partition);
+ sourceRuntime = getIntakeRuntime(feedSubscribableRuntimeId);
+ if (sourceRuntime == null) {
+ throw new HyracksDataException("Source intake task not found for source feed id "
+ + sourceFeedId);
+ }
+ nodePushable = new FeedCollectOperatorNodePushable(ctx, sourceFeedId, connectionId,
+ feedPolicyProperties, partition, nPartitions, sourceRuntime);
+
+ } catch (Exception exception) {
+ if (LOGGER.isLoggable(Level.SEVERE)) {
+ LOGGER.severe("Initialization of the feed adaptor failed with exception " + exception);
+ }
+ throw new HyracksDataException("Initialization of the feed adapter failed", exception);
+ }
+ break;
+ case SOURCE_FEED_COMPUTE_STAGE:
+ SubscribableFeedRuntimeId feedSubscribableRuntimeId = new SubscribableFeedRuntimeId(sourceFeedId,
+ FeedRuntimeType.COMPUTE, partition);
+ sourceRuntime = (ISubscribableRuntime) subscriptionManager
+ .getSubscribableRuntime(feedSubscribableRuntimeId);
+ if (sourceRuntime == null) {
+ throw new HyracksDataException("Source compute task not found for source feed id " + sourceFeedId
+ + " " + FeedRuntimeType.COMPUTE + "[" + partition + "]");
+ }
+ nodePushable = new FeedCollectOperatorNodePushable(ctx, sourceFeedId, connectionId,
+ feedPolicyProperties, partition, nPartitions, sourceRuntime);
+ break;
+ }
+ return nodePushable;
+ }
+
+ public FeedConnectionId getFeedConnectionId() {
+ return connectionId;
+ }
+
+ public Map<String, String> getFeedPolicyProperties() {
+ return feedPolicyProperties;
+ }
+
+ public IAType getOutputType() {
+ return outputType;
+ }
+
+ public RecordDescriptor getRecordDescriptor() {
+ return recordDescriptors[0];
+ }
+
+ public FeedId getSourceFeedId() {
+ return sourceFeedId;
+ }
+
+ private IngestionRuntime getIntakeRuntime(SubscribableFeedRuntimeId subscribableRuntimeId) {
+ int waitCycleCount = 0;
+ ISubscribableRuntime ingestionRuntime = subscriptionManager.getSubscribableRuntime(subscribableRuntimeId);
+ while (ingestionRuntime == null && waitCycleCount < 10) {
+ try {
+ Thread.sleep(2000);
+ waitCycleCount++;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("waiting to obtain ingestion runtime for subscription " + subscribableRuntimeId);
+ }
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ break;
+ }
+ ingestionRuntime = subscriptionManager.getSubscribableRuntime(subscribableRuntimeId);
+ }
+ return (IngestionRuntime) ingestionRuntime;
+ }
+
+ public ConnectionLocation getSubscriptionLocation() {
+ return subscriptionLocation;
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedCollectOperatorNodePushable.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedCollectOperatorNodePushable.java
new file mode 100644
index 0000000..48d1dc6
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedCollectOperatorNodePushable.java
@@ -0,0 +1,204 @@
+/*
+ * 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.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.feeds.CollectionRuntime;
+import edu.uci.ics.asterix.common.feeds.FeedCollectRuntimeInputHandler;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedFrameCollector.State;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeInputHandler;
+import edu.uci.ics.asterix.common.feeds.SubscribableFeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedOperatorOutputSideHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.Mode;
+import edu.uci.ics.asterix.common.feeds.api.ISubscribableRuntime;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+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 FeedCollectOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+
+ private static Logger LOGGER = Logger.getLogger(FeedCollectOperatorNodePushable.class.getName());
+
+ private final int partition;
+ private final FeedConnectionId connectionId;
+ private final Map<String, String> feedPolicy;
+ private final FeedPolicyAccessor policyAccessor;
+ private final IFeedManager feedManager;
+ private final ISubscribableRuntime sourceRuntime;
+ private final IHyracksTaskContext ctx;
+ private final int nPartitions;
+
+ private RecordDescriptor outputRecordDescriptor;
+ private FeedRuntimeInputHandler inputSideHandler;
+ private CollectionRuntime collectRuntime;
+
+ public FeedCollectOperatorNodePushable(IHyracksTaskContext ctx, FeedId sourceFeedId,
+ FeedConnectionId feedConnectionId, Map<String, String> feedPolicy, int partition, int nPartitions,
+ ISubscribableRuntime sourceRuntime) {
+ this.ctx = ctx;
+ this.partition = partition;
+ this.nPartitions = nPartitions;
+ this.connectionId = feedConnectionId;
+ this.sourceRuntime = sourceRuntime;
+ this.feedPolicy = feedPolicy;
+ policyAccessor = new FeedPolicyAccessor(feedPolicy);
+ IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+ .getApplicationContext().getApplicationObject();
+ this.feedManager = runtimeCtx.getFeedManager();
+ }
+
+ @Override
+ public void initialize() throws HyracksDataException {
+ try {
+ outputRecordDescriptor = recordDesc;
+ FeedRuntimeType sourceRuntimeType = ((SubscribableFeedRuntimeId) sourceRuntime.getRuntimeId())
+ .getFeedRuntimeType();
+ switch (sourceRuntimeType) {
+ case INTAKE:
+ handleCompleteConnection();
+ break;
+ case COMPUTE:
+ handlePartialConnection();
+ break;
+ default:
+ throw new IllegalStateException("Invalid source type " + sourceRuntimeType);
+ }
+
+ State state = collectRuntime.waitTillCollectionOver();
+ if (state.equals(State.FINISHED)) {
+ feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId,
+ collectRuntime.getRuntimeId());
+ writer.close();
+ inputSideHandler.close();
+ } else if (state.equals(State.HANDOVER)) {
+ inputSideHandler.setMode(Mode.STALL);
+ writer.close();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Ending Collect Operator, the input side handler is now in " + Mode.STALL
+ + " and the output writer " + writer + " has been closed ");
+ }
+ }
+ } catch (InterruptedException ie) {
+ handleInterruptedException(ie);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ private void handleCompleteConnection() throws Exception {
+ FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.COLLECT, partition,
+ FeedRuntimeId.DEFAULT_OPERAND_ID);
+ collectRuntime = (CollectionRuntime) feedManager.getFeedConnectionManager().getFeedRuntime(connectionId,
+ runtimeId);
+ if (collectRuntime == null) {
+ beginNewFeed(runtimeId);
+ } else {
+ reviveOldFeed();
+ }
+ }
+
+ private void beginNewFeed(FeedRuntimeId runtimeId) throws Exception {
+ writer.open();
+ IFrameWriter outputSideWriter = writer;
+ if (((SubscribableFeedRuntimeId) sourceRuntime.getRuntimeId()).getFeedRuntimeType().equals(
+ FeedRuntimeType.COMPUTE)) {
+ outputSideWriter = new CollectTransformFeedFrameWriter(ctx, writer, sourceRuntime, outputRecordDescriptor,
+ connectionId);
+ this.recordDesc = sourceRuntime.getRecordDescriptor();
+ }
+
+ FrameTupleAccessor tupleAccessor = new FrameTupleAccessor(recordDesc);
+ inputSideHandler = new FeedCollectRuntimeInputHandler(ctx, connectionId, runtimeId, outputSideWriter, policyAccessor,
+ false, tupleAccessor, recordDesc,
+ feedManager, nPartitions);
+
+ collectRuntime = new CollectionRuntime(connectionId, runtimeId, inputSideHandler, outputSideWriter,
+ sourceRuntime, feedPolicy);
+ feedManager.getFeedConnectionManager().registerFeedRuntime(connectionId, collectRuntime);
+ sourceRuntime.subscribeFeed(policyAccessor, collectRuntime);
+ }
+
+ private void reviveOldFeed() throws HyracksDataException {
+ writer.open();
+ collectRuntime.getFrameCollector().setState(State.ACTIVE);
+ inputSideHandler = collectRuntime.getInputHandler();
+
+ IFrameWriter innerWriter = inputSideHandler.getCoreOperator();
+ if (innerWriter instanceof CollectTransformFeedFrameWriter) {
+ ((CollectTransformFeedFrameWriter) innerWriter).reset(this.writer);
+ } else {
+ inputSideHandler.setCoreOperator(writer);
+ }
+
+ inputSideHandler.setMode(Mode.PROCESS);
+ }
+
+ private void handlePartialConnection() throws Exception {
+ FeedRuntimeId runtimeId = new FeedRuntimeId(FeedRuntimeType.COMPUTE_COLLECT, partition,
+ FeedRuntimeId.DEFAULT_OPERAND_ID);
+ writer.open();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Beginning new feed (from existing partial connection):" + connectionId);
+ }
+ IFeedOperatorOutputSideHandler wrapper = new CollectTransformFeedFrameWriter(ctx, writer, sourceRuntime,
+ outputRecordDescriptor, connectionId);
+
+ inputSideHandler = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, wrapper, policyAccessor, false,
+ new FrameTupleAccessor(recordDesc), recordDesc, feedManager,
+ nPartitions);
+
+ collectRuntime = new CollectionRuntime(connectionId, runtimeId, inputSideHandler, wrapper, sourceRuntime,
+ feedPolicy);
+ feedManager.getFeedConnectionManager().registerFeedRuntime(connectionId, collectRuntime);
+ recordDesc = sourceRuntime.getRecordDescriptor();
+ sourceRuntime.subscribeFeed(policyAccessor, collectRuntime);
+ }
+
+ private void handleInterruptedException(InterruptedException ie) throws HyracksDataException {
+ if (policyAccessor.continueOnHardwareFailure()) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Continuing on failure as per feed policy, switching to " + Mode.STALL
+ + " until failure is resolved");
+ }
+ inputSideHandler.setMode(Mode.STALL);
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Failure during feed ingestion. Deregistering feed runtime " + collectRuntime
+ + " as feed is not configured to handle failures");
+ }
+ feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId, collectRuntime.getRuntimeId());
+ writer.close();
+ throw new HyracksDataException(ie);
+ }
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedConnectionManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedConnectionManager.java
new file mode 100644
index 0000000..eefb576
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedConnectionManager.java
@@ -0,0 +1,105 @@
+/*
+ * 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.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 edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedConnectionManager;
+
+/**
+ * An implementation of the IFeedManager interface.
+ * Provider necessary central repository for registering/retrieving
+ * artifacts/services associated with a feed.
+ */
+public class FeedConnectionManager implements IFeedConnectionManager {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedConnectionManager.class.getName());
+
+ private Map<FeedConnectionId, FeedRuntimeManager> feedRuntimeManagers = new HashMap<FeedConnectionId, FeedRuntimeManager>();
+ private final String nodeId;
+
+ public FeedConnectionManager(String nodeId) {
+ this.nodeId = nodeId;
+ }
+
+ public FeedRuntimeManager getFeedRuntimeManager(FeedConnectionId feedId) {
+ return feedRuntimeManagers.get(feedId);
+ }
+
+ @Override
+ public void deregisterFeed(FeedConnectionId feedId) {
+ try {
+ FeedRuntimeManager mgr = feedRuntimeManagers.get(feedId);
+ if (mgr != null) {
+ mgr.close();
+ feedRuntimeManagers.remove(feedId);
+ }
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Exception in closing feed runtime" + e.getMessage());
+ }
+ }
+
+ }
+
+ @Override
+ public synchronized void registerFeedRuntime(FeedConnectionId connectionId, FeedRuntime feedRuntime)
+ throws Exception {
+ FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(connectionId);
+ if (runtimeMgr == null) {
+ runtimeMgr = new FeedRuntimeManager(connectionId, this);
+ feedRuntimeManagers.put(connectionId, runtimeMgr);
+ }
+ runtimeMgr.registerFeedRuntime(feedRuntime.getRuntimeId(), feedRuntime);
+ }
+
+ @Override
+ public void deRegisterFeedRuntime(FeedConnectionId connectionId, FeedRuntimeId feedRuntimeId) {
+ FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(connectionId);
+ if (runtimeMgr != null) {
+ runtimeMgr.deregisterFeedRuntime(feedRuntimeId);
+ }
+ }
+
+ @Override
+ public FeedRuntime getFeedRuntime(FeedConnectionId connectionId, FeedRuntimeId feedRuntimeId) {
+ FeedRuntimeManager runtimeMgr = feedRuntimeManagers.get(connectionId);
+ return runtimeMgr != null ? runtimeMgr.getFeedRuntime(feedRuntimeId) : null;
+ }
+
+ @Override
+ public String toString() {
+ return "FeedManager " + "[" + nodeId + "]";
+ }
+
+ @Override
+ public List<FeedRuntimeId> getRegisteredRuntimes() {
+ List<FeedRuntimeId> runtimes = new ArrayList<FeedRuntimeId>();
+ for (Entry<FeedConnectionId, FeedRuntimeManager> entry : feedRuntimeManagers.entrySet()) {
+ runtimes.addAll(entry.getValue().getFeedRuntimes());
+ }
+ return runtimes;
+ }
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedFrameTupleDecorator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedFrameTupleDecorator.java
new file mode 100644
index 0000000..d131b95
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedFrameTupleDecorator.java
@@ -0,0 +1,90 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import edu.uci.ics.asterix.builders.IARecordBuilder;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedConstants.StatisticsConstants;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AInt64;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.AMutableString;
+import edu.uci.ics.asterix.om.base.AString;
+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;
+
+public class FeedFrameTupleDecorator {
+
+ private AMutableString aString = new AMutableString("");
+ private AMutableInt64 aInt64 = new AMutableInt64(0);
+ private AMutableInt32 aInt32 = new AMutableInt32(0);
+ private AtomicInteger tupleId;
+
+ @SuppressWarnings("unchecked")
+ private static ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.ASTRING);
+ @SuppressWarnings("unchecked")
+ private static ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT32);
+ @SuppressWarnings("unchecked")
+ private static ISerializerDeserializer<AInt64> int64Serde = AqlSerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(BuiltinType.AINT64);
+
+ private final int partition;
+ private final ArrayBackedValueStorage attrNameStorage;
+ private final ArrayBackedValueStorage attrValueStorage;
+
+ public FeedFrameTupleDecorator(int partition) {
+ this.tupleId = new AtomicInteger(0);
+ this.partition = partition;
+ this.attrNameStorage = new ArrayBackedValueStorage();
+ this.attrValueStorage = new ArrayBackedValueStorage();
+ }
+
+ public void addLongAttribute(String attrName, long attrValue, IARecordBuilder recordBuilder)
+ throws HyracksDataException, AsterixException {
+ attrNameStorage.reset();
+ aString.setValue(attrName);
+ stringSerde.serialize(aString, attrNameStorage.getDataOutput());
+
+ attrValueStorage.reset();
+ aInt64.setValue(attrValue);
+ int64Serde.serialize(aInt64, attrValueStorage.getDataOutput());
+
+ recordBuilder.addField(attrNameStorage, attrValueStorage);
+ }
+
+ public void addIntegerAttribute(String attrName, int attrValue, IARecordBuilder recordBuilder)
+ throws HyracksDataException, AsterixException {
+ attrNameStorage.reset();
+ aString.setValue(attrName);
+ stringSerde.serialize(aString, attrNameStorage.getDataOutput());
+
+ attrValueStorage.reset();
+ aInt32.setValue(attrValue);
+ int32Serde.serialize(aInt32, attrValueStorage.getDataOutput());
+
+ recordBuilder.addField(attrNameStorage, attrValueStorage);
+ }
+
+ public void addTupleId(IARecordBuilder recordBuilder) throws HyracksDataException, AsterixException {
+ addIntegerAttribute(StatisticsConstants.INTAKE_TUPLEID, tupleId.incrementAndGet(), recordBuilder);
+ }
+
+ public void addIntakePartition(IARecordBuilder recordBuilder) throws HyracksDataException, AsterixException {
+ addIntegerAttribute(StatisticsConstants.INTAKE_PARTITION, partition, recordBuilder);
+ }
+
+ public void addIntakeTimestamp(IARecordBuilder recordBuilder) throws HyracksDataException, AsterixException {
+ addLongAttribute(StatisticsConstants.INTAKE_TIMESTAMP, System.currentTimeMillis(), recordBuilder);
+ }
+
+ public void addStoreTimestamp(IARecordBuilder recordBuilder) throws HyracksDataException, AsterixException {
+ addLongAttribute(StatisticsConstants.STORE_TIMESTAMP, System.currentTimeMillis(), recordBuilder);
+ }
+
+}
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
deleted file mode 100644
index 899da77..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedFrameWriter.java
+++ /dev/null
@@ -1,385 +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.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.common.feeds.FeedConnectionId;
-import edu.uci.ics.asterix.common.feeds.FeedMessageService;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
-import edu.uci.ics.asterix.common.feeds.IFeedManager;
-import edu.uci.ics.asterix.common.feeds.SuperFeedManager;
-import edu.uci.ics.asterix.common.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; // 5 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, it measure the
- * throughput as observed on the output channel of the associated operator.
- */
- private HealthMonitor healthMonitor;
-
- /**
- * A Timer instance for managing 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. This mode is adopted
- * during failure recovery.
- */
- STORE
- }
-
- public FeedFrameWriter(IFrameWriter writer, IOperatorNodePushable nodePushable, FeedConnectionId feedId,
- FeedPolicyEnforcer policyEnforcer, String nodeId, FeedRuntimeType feedRuntimeType, int partition,
- FrameTupleAccessor fta, IFeedManager feedManager) {
- this.writer = writer;
- this.mode = Mode.FORWARD;
- this.nodePushable = nodePushable;
- this.reportHealth = policyEnforcer.getFeedPolicyAccessor().collectStatistics();
- if (reportHealth) {
- timer = new Timer();
- healthMonitor = new HealthMonitor(feedId, nodeId, feedRuntimeType, partition, timer, fta, feedManager);
- 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.notifyStartFrameFlushActivity();
- writer.nextFrame(buffer);
- healthMonitor.notifyFinishFrameFlushActivity();
- } else {
- writer.nextFrame(buffer);
- }
- } catch (Exception e) {
- e.printStackTrace();
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe("Unable to write frame " + " on behalf of " + nodePushable.getDisplayName()
- + ":\n" + e);
- }
- }
- if (frames.size() > 0) {
- for (ByteBuffer buf : frames) {
- writer.nextFrame(buf);
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Flushed old frame (from previous failed execution) : " + buf
- + " on behalf of " + nodePushable.getDisplayName());
- }
- }
- frames.clear();
- }
- break;
- case STORE:
-
- /* TODO:
- * Limit the in-memory space utilized during the STORE mode. The limit (expressed in bytes)
- * is a parameter specified as part of the feed ingestion policy. Below is a basic implemenation
- * that allocates a buffer on demand.
- * */
-
- 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.getDisplayName());
- }
- break;
- }
- }
-
- /**
- * Detects if the operator is unable to push a frame downstream
- * within a threshold period of time. In addition, it measure the
- * throughput as observed on the output channel of the associated operator.
- */
- private static class HealthMonitor extends TimerTask {
-
- private static final String EOL = "\n";
-
- private long startTime = -1;
- private FramePushState state;
- private AtomicLong numTuplesInInterval = new AtomicLong(0);
- private boolean collectThroughput;
- private FeedMessageService mesgService;
-
- private final FeedConnectionId feedId;
- private final String nodeId;
- private final FeedRuntimeType feedRuntimeType;
- private final int partition;
- private final long period;
- private final FrameTupleAccessor fta;
- private final IFeedManager feedManager;
-
- public HealthMonitor(FeedConnectionId feedId, String nodeId, FeedRuntimeType feedRuntimeType, int partition,
- Timer timer, FrameTupleAccessor fta, IFeedManager feedManager) {
- this.state = FramePushState.INTIALIZED;
- this.feedId = feedId;
- this.nodeId = nodeId;
- this.feedRuntimeType = feedRuntimeType;
- this.partition = partition;
- this.period = FLUSH_THRESHOLD_TIME;
- this.collectThroughput = feedRuntimeType.equals(FeedRuntimeType.INGESTION);
- this.fta = fta;
- this.feedManager = feedManager;
- }
-
- public void notifyStartFrameFlushActivity() {
- 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 = feedRuntimeType.equals(FeedRuntimeType.INGESTION);
- }
-
- public void notifyFinishFrameFlushActivity() {
- state = FramePushState.WAITNG_FOR_NEXT_FRAME;
- numTuplesInInterval.set(numTuplesInInterval.get() + fta.getTupleCount());
- }
-
- @Override
- public void run() {
- if (state.equals(FramePushState.WAITING_FOR_FLUSH_COMPLETION)) {
- long currentTime = System.currentTimeMillis();
- if (currentTime - startTime > FLUSH_THRESHOLD_TIME) {
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe("Congestion reported by " + feedRuntimeType + " [" + partition + "]");
- }
- sendReportToSuperFeedManager(currentTime - startTime, FeedReportMessageType.CONGESTION,
- System.currentTimeMillis());
- }
- }
- if (collectThroughput) {
- int instantTput = (int) Math.ceil((((double) numTuplesInInterval.get() * 1000) / period));
- sendReportToSuperFeedManager(instantTput, FeedReportMessageType.THROUGHPUT, System.currentTimeMillis());
- }
- numTuplesInInterval.set(0);
- }
-
- private void sendReportToSuperFeedManager(long value, SuperFeedManager.FeedReportMessageType mesgType,
- long timestamp) {
- if (mesgService == null) {
- waitTillMessageServiceIsUp();
- }
- String feedRep = feedId.getDataverse() + ":" + feedId.getFeedName() + ":" + feedId.getDatasetName();
- String message = mesgType.name().toLowerCase() + FeedMessageService.MessageSeparator + feedRep
- + FeedMessageService.MessageSeparator + feedRuntimeType + FeedMessageService.MessageSeparator
- + partition + FeedMessageService.MessageSeparator + value + FeedMessageService.MessageSeparator
- + nodeId + FeedMessageService.MessageSeparator + timestamp + FeedMessageService.MessageSeparator
- + EOL;
- try {
- mesgService.sendMessage(message);
- } catch (IOException ioe) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to send feed report to Super Feed Manager for feed " + feedId + " "
- + feedRuntimeType + "[" + partition + "]");
- }
- }
- }
-
- private void waitTillMessageServiceIsUp() {
- while (mesgService == null) {
- mesgService = feedManager.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() {
- // cancel the timer task to avoid future execution.
- 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) {
- if (!healthMonitor.feedRuntimeType.equals(FeedRuntimeType.INGESTION)) {
- healthMonitor.deactivate();
- } else {
- 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
index 2b05989..1282f85 100644
--- 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
@@ -15,194 +15,117 @@
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.common.api.IAsterixAppRuntimeContext;
-import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeId;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
-import edu.uci.ics.asterix.common.feeds.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.feeds.IngestionRuntime;
+import edu.uci.ics.asterix.common.feeds.SubscribableFeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedSubscriptionManager;
+import edu.uci.ics.asterix.metadata.entities.PrimaryFeed;
import edu.uci.ics.asterix.metadata.functions.ExternalLibraryManager;
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 adapter for retrieving data from the external
- * data source.
+ * An operator responsible for establishing connection with external data source and parsing,
+ * translating the received content.It uses an instance of feed adaptor to perform these functions.
*/
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 adapter */
- private final IAType outputType;
+ /** The unique identifier of the feed that is being ingested. **/
+ private final FeedId feedId;
- /** unique identifier for a feed instance. */
- private final FeedConnectionId feedId;
+ private final FeedPolicyAccessor policyAccessor;
- /** Map representation of policy parameters */
- private final Map<String, String> feedPolicy;
-
- /** The adapter factory that is used to create an instance of the feed adapter **/
- private IAdapterFactory adapterFactory;
-
- /** The (singleton) instance of IFeedManager **/
- private IFeedManager feedManager;
+ /** The adaptor factory that is used to create an instance of the feed adaptor **/
+ private IFeedAdapterFactory adaptorFactory;
/** The library that contains the adapter in use. **/
- private String adapterLibraryName;
+ private String adaptorLibraryName;
/**
* The adapter factory class that is used to create an instance of the feed adapter.
* This value is used only in the case of external adapters.
**/
- private String adapterFactoryClassName;
+ private String adaptorFactoryClassName;
/** The configuration parameters associated with the adapter. **/
- private Map<String, String> adapterConfiguration;
+ private Map<String, String> adaptorConfiguration;
private ARecordType adapterOutputType;
- public FeedIntakeOperatorDescriptor(JobSpecification spec, FeedConnectionId feedId, IAdapterFactory adapterFactory,
- ARecordType atype, RecordDescriptor rDesc, Map<String, String> feedPolicy) {
+ public FeedIntakeOperatorDescriptor(JobSpecification spec, PrimaryFeed primaryFeed,
+ IFeedAdapterFactory adapterFactory, ARecordType adapterOutputType, FeedPolicyAccessor policyAccessor) {
super(spec, 0, 1);
- recordDescriptors[0] = rDesc;
- this.adapterFactory = adapterFactory;
- this.outputType = atype;
- this.feedId = feedId;
- this.feedPolicy = feedPolicy;
+ this.feedId = new FeedId(primaryFeed.getDataverseName(), primaryFeed.getFeedName());
+ this.adaptorFactory = adapterFactory;
+ this.adapterOutputType = adapterOutputType;
+ this.policyAccessor = policyAccessor;
}
- public FeedIntakeOperatorDescriptor(JobSpecification spec, FeedConnectionId feedId, String adapterLibraryName,
- String adapterFactoryClassName, Map<String, String> configuration, ARecordType atype,
- RecordDescriptor rDesc, Map<String, String> feedPolicy) {
+ public FeedIntakeOperatorDescriptor(JobSpecification spec, PrimaryFeed primaryFeed, String adapterLibraryName,
+ String adapterFactoryClassName, ARecordType adapterOutputType, FeedPolicyAccessor policyAccessor) {
super(spec, 0, 1);
- recordDescriptors[0] = rDesc;
- this.adapterFactoryClassName = adapterFactoryClassName;
- this.adapterConfiguration = configuration;
- this.adapterLibraryName = adapterLibraryName;
- this.outputType = atype;
- this.feedId = feedId;
- this.feedPolicy = feedPolicy;
+ this.feedId = new FeedId(primaryFeed.getDataverseName(), primaryFeed.getFeedName());
+ this.adaptorFactoryClassName = adapterFactoryClassName;
+ this.adaptorLibraryName = adapterLibraryName;
+ this.adaptorConfiguration = primaryFeed.getAdaptorConfiguration();
+ this.adapterOutputType = adapterOutputType;
+ this.policyAccessor = policyAccessor;
}
+ @Override
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
- throws HyracksDataException {
- IFeedAdapter adapter = null;
- FeedRuntimeId feedRuntimeId = new FeedRuntimeId(feedId, FeedRuntimeType.INGESTION, partition);
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
.getApplicationContext().getApplicationObject();
- this.feedManager = runtimeCtx.getFeedManager();
- IngestionRuntime ingestionRuntime = (IngestionRuntime) feedManager.getFeedRuntime(feedRuntimeId);
- try {
- if (ingestionRuntime == null) {
- // create an instance of a feed adapter to ingest data.
- adapter = createAdapter(ctx, partition);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Beginning new feed:" + feedId);
- }
- } else {
- // retrieve the instance of the feed adapter used in previous failed execution.
- adapter = ((IngestionRuntime) ingestionRuntime).getAdapterRuntimeManager().getFeedAdapter();
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Resuming old feed:" + feedId);
- }
+ IFeedSubscriptionManager feedSubscriptionManager = runtimeCtx.getFeedManager().getFeedSubscriptionManager();
+ SubscribableFeedRuntimeId feedIngestionId = new SubscribableFeedRuntimeId(feedId, FeedRuntimeType.INTAKE,
+ partition);
+ IngestionRuntime ingestionRuntime = (IngestionRuntime) feedSubscriptionManager
+ .getSubscribableRuntime(feedIngestionId);
+ if (adaptorFactory == null) {
+ try {
+ adaptorFactory = createExtenralAdapterFactory(ctx, partition);
+ } catch (Exception exception) {
+ throw new HyracksDataException(exception);
}
- } catch (Exception exception) {
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe("Initialization of the feed adapter failed with exception " + exception);
- }
- throw new HyracksDataException("Initialization of the feed adapter failed", exception);
+
}
- return new FeedIntakeOperatorNodePushable(ctx, feedId, adapter, feedPolicy, partition, ingestionRuntime);
+ return new FeedIntakeOperatorNodePushable(ctx, feedId, adaptorFactory, partition, ingestionRuntime,
+ policyAccessor);
}
- public FeedConnectionId getFeedId() {
- return feedId;
- }
-
- public Map<String, String> getFeedPolicy() {
- return feedPolicy;
- }
-
- public IAdapterFactory getAdapterFactory() {
+ private IFeedAdapterFactory createExtenralAdapterFactory(IHyracksTaskContext ctx, int partition) throws Exception {
+ IFeedAdapterFactory adapterFactory = null;
+ ClassLoader classLoader = ExternalLibraryManager.getLibraryClassLoader(feedId.getDataverse(),
+ adaptorLibraryName);
+ if (classLoader != null) {
+ adapterFactory = ((IFeedAdapterFactory) (classLoader.loadClass(adaptorFactoryClassName).newInstance()));
+ adapterFactory.configure(adaptorConfiguration, adapterOutputType);
+ } else {
+ String message = "Unable to create adapter as class loader not configured for library "
+ + adaptorLibraryName + " in dataverse " + feedId.getDataverse();
+ LOGGER.severe(message);
+ throw new IllegalArgumentException(message);
+ }
return adapterFactory;
}
- public IAType getOutputType() {
- return outputType;
+ public FeedId getFeedId() {
+ return feedId;
}
- public RecordDescriptor getRecordDescriptor() {
- return recordDescriptors[0];
- }
-
- private IFeedAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
- IFeedAdapter feedAdapter = null;
- if (adapterFactory != null) {
- feedAdapter = (IFeedAdapter) adapterFactory.createAdapter(ctx, partition);
- } else {
- ClassLoader classLoader = ExternalLibraryManager.getLibraryClassLoader(feedId.getDataverse(),
- adapterLibraryName);
- if (classLoader != null) {
- IAdapterFactory adapterFactory = ((IAdapterFactory) (classLoader.loadClass(adapterFactoryClassName)
- .newInstance()));
-
- switch (adapterFactory.getAdapterType()) {
- case TYPED: {
- ((ITypedAdapterFactory) adapterFactory).configure(adapterConfiguration);
- feedAdapter = (IFeedAdapter) ((ITypedAdapterFactory) adapterFactory).createAdapter(ctx,
- partition);
- }
- break;
- case GENERIC: {
- String outputTypeName = adapterConfiguration.get(IGenericAdapterFactory.KEY_TYPE_NAME);
- if (outputTypeName == null) {
- throw new IllegalArgumentException(
- "You must specify the datatype associated with the incoming data. Datatype is specified by the "
- + IGenericAdapterFactory.KEY_TYPE_NAME + " configuration parameter");
- }
- ((IGenericAdapterFactory) adapterFactory).configure(adapterConfiguration,
- (ARecordType) adapterOutputType);
- ((IGenericAdapterFactory) adapterFactory).createAdapter(ctx, partition);
- }
- break;
- }
-
- feedAdapter = (IFeedAdapter) adapterFactory.createAdapter(ctx, partition);
- } else {
- String message = "Unable to create adapter as class loader not configured for library "
- + adapterLibraryName + " in dataverse " + feedId.getDataverse();
- if (LOGGER.isLoggable(Level.SEVERE)) {
- LOGGER.severe(message);
- }
- throw new IllegalArgumentException(message);
-
- }
- }
- return feedAdapter;
- }
-
- public String getAdapterLibraryName() {
- return adapterLibraryName;
- }
-
- public String getAdapterFactoryClassName() {
- return adapterFactoryClassName;
- }
-
- public Map<String, String> getAdapterConfiguration() {
- return adapterConfiguration;
- }
}
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
index a6b97eb..3256420 100644
--- 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
@@ -14,133 +14,196 @@
*/
package edu.uci.ics.asterix.metadata.feeds;
-import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.LinkedBlockingQueue;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
import java.util.logging.Level;
import java.util.logging.Logger;
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
-import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
-import edu.uci.ics.asterix.common.feeds.FeedRuntimeManager;
-import edu.uci.ics.asterix.common.feeds.IFeedManager;
-import edu.uci.ics.asterix.metadata.feeds.AdapterRuntimeManager.State;
-import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter.DataExchangeMode;
+import edu.uci.ics.asterix.common.feeds.CollectionRuntime;
+import edu.uci.ics.asterix.common.feeds.DistributeFeedFrameWriter;
+import edu.uci.ics.asterix.common.feeds.FeedId;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.feeds.IngestionRuntime;
+import edu.uci.ics.asterix.common.feeds.SubscribableFeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.api.IAdapterRuntimeManager;
+import edu.uci.ics.asterix.common.feeds.api.IAdapterRuntimeManager.State;
+import edu.uci.ics.asterix.common.feeds.api.IFeedAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedSubscriptionManager;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
+import edu.uci.ics.asterix.common.feeds.api.ISubscriberRuntime;
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}
+ * The runtime for @see{FeedIntakeOperationDescriptor}.
+ * Provides the core functionality to set up the artifacts for ingestion of a feed.
+ * The artifacts are lazily activated when a feed receives a subscription request.
*/
public class FeedIntakeOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
private static Logger LOGGER = Logger.getLogger(FeedIntakeOperatorNodePushable.class.getName());
+ private final FeedId feedId;
private final int partition;
- private final FeedConnectionId feedId;
- private final LinkedBlockingQueue<IFeedMessage> inbox;
- private final Map<String, String> feedPolicy;
- private final FeedPolicyEnforcer policyEnforcer;
- private final String nodeId;
- private final FrameTupleAccessor fta;
+ private final IFeedSubscriptionManager feedSubscriptionManager;
private final IFeedManager feedManager;
+ private final IHyracksTaskContext ctx;
+ private final IFeedAdapterFactory adapterFactory;
+ private final FeedPolicyAccessor policyAccessor;
- private FeedRuntime ingestionRuntime;
+ private IngestionRuntime ingestionRuntime;
private IFeedAdapter adapter;
- private FeedFrameWriter feedFrameWriter;
+ private IIntakeProgressTracker tracker;
+ private DistributeFeedFrameWriter feedFrameWriter;
- public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId feedId, IFeedAdapter adapter,
- Map<String, String> feedPolicy, int partition, IngestionRuntime ingestionRuntime) {
- this.adapter = adapter;
- this.partition = partition;
+ public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, FeedId feedId, IFeedAdapterFactory adapterFactory,
+ int partition, IngestionRuntime ingestionRuntime, FeedPolicyAccessor policyAccessor) {
+ this.ctx = ctx;
this.feedId = feedId;
+ this.partition = partition;
this.ingestionRuntime = ingestionRuntime;
- inbox = new LinkedBlockingQueue<IFeedMessage>();
- this.feedPolicy = feedPolicy;
- policyEnforcer = new FeedPolicyEnforcer(feedId, feedPolicy);
- nodeId = ctx.getJobletContext().getApplicationContext().getNodeId();
- fta = new FrameTupleAccessor(recordDesc);
+ this.adapterFactory = adapterFactory;
IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
.getApplicationContext().getApplicationObject();
+ this.feedSubscriptionManager = runtimeCtx.getFeedManager().getFeedSubscriptionManager();
this.feedManager = runtimeCtx.getFeedManager();
+ this.policyAccessor = policyAccessor;
}
@Override
public void initialize() throws HyracksDataException {
-
- AdapterRuntimeManager adapterRuntimeMgr = null;
+ IAdapterRuntimeManager adapterRuntimeManager = null;
try {
if (ingestionRuntime == null) {
- feedFrameWriter = new FeedFrameWriter(writer, this, feedId, policyEnforcer, nodeId,
- FeedRuntimeType.INGESTION, partition, fta, feedManager);
- adapterRuntimeMgr = new AdapterRuntimeManager(feedId, adapter, feedFrameWriter, partition, inbox,
- feedManager);
-
- if (adapter.getDataExchangeMode().equals(DataExchangeMode.PULL)
- && adapter instanceof IPullBasedFeedAdapter) {
- ((IPullBasedFeedAdapter) 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.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.getFeedRuntimeManager(feedId);
try {
- runtimeMgr.close(false);
- } catch (IOException ioe) {
+ adapter = (IFeedAdapter) adapterFactory.createAdapter(ctx, partition);
+ if (adapterFactory.isRecordTrackingEnabled()) {
+ tracker = adapterFactory.createIntakeProgressTracker();
+ }
+ } catch (Exception e) {
+ LOGGER.severe("Unable to create adapter : " + adapterFactory.getName() + "[" + partition + "]"
+ + " Exception " + e);
+ throw new HyracksDataException(e);
+ }
+ FrameTupleAccessor fta = new FrameTupleAccessor(recordDesc);
+ feedFrameWriter = new DistributeFeedFrameWriter(ctx, feedId, writer, FeedRuntimeType.INTAKE, partition, fta,
+ feedManager);
+ adapterRuntimeManager = new AdapterRuntimeManager(feedId, adapter, tracker, feedFrameWriter, partition);
+ SubscribableFeedRuntimeId runtimeId = new SubscribableFeedRuntimeId(feedId, FeedRuntimeType.INTAKE,
+ partition);
+ ingestionRuntime = new IngestionRuntime(feedId, runtimeId, feedFrameWriter, recordDesc,
+ adapterRuntimeManager);
+ feedSubscriptionManager.registerFeedSubscribableRuntime(ingestionRuntime);
+ feedFrameWriter.open();
+ } else {
+ if (ingestionRuntime.getAdapterRuntimeManager().getState().equals(State.INACTIVE_INGESTION)) {
+ ingestionRuntime.getAdapterRuntimeManager().setState(State.ACTIVE_INGESTION);
+ adapter = ingestionRuntime.getAdapterRuntimeManager().getFeedAdapter();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(" Switching to " + State.ACTIVE_INGESTION + " for ingestion runtime "
+ + ingestionRuntime);
+ LOGGER.info(" Adaptor " + adapter.getClass().getName() + "[" + partition + "]"
+ + " connected to backend for feed " + feedId);
+ }
+ feedFrameWriter = (DistributeFeedFrameWriter) ingestionRuntime.getFeedFrameWriter();
+ } else {
+ String message = "Feed Ingestion Runtime for feed " + feedId
+ + " is already registered and is active!.";
+ LOGGER.severe(message);
+ throw new IllegalStateException(message);
+ }
+ }
+
+ waitTillIngestionIsOver(adapterRuntimeManager);
+ feedSubscriptionManager.deregisterFeedSubscribableRuntime((SubscribableFeedRuntimeId) ingestionRuntime
+ .getRuntimeId());
+ if (adapterRuntimeManager.getState().equals(IAdapterRuntimeManager.State.FAILED_INGESTION)) {
+ throw new HyracksDataException("Unable to ingest data");
+ }
+
+ } catch (InterruptedException ie) {
+ /*
+ * An Interrupted Exception is thrown if the Intake job cannot progress further due to failure of another node involved in the Hyracks job.
+ * As the Intake job involves only the intake operator, the exception is indicative of a failure at the sibling intake operator location.
+ * The surviving intake partitions must continue to live and receive data from the external source.
+ */
+ List<ISubscriberRuntime> subscribers = ingestionRuntime.getSubscribers();
+ FeedPolicyAccessor policyAccessor = new FeedPolicyAccessor(new HashMap<String, String>());
+ boolean needToHandleFailure = false;
+ List<ISubscriberRuntime> failingSubscribers = new ArrayList<ISubscriberRuntime>();
+ for (ISubscriberRuntime subscriber : subscribers) {
+ policyAccessor.reset(subscriber.getFeedPolicy());
+ if (!policyAccessor.continueOnHardwareFailure()) {
+ failingSubscribers.add(subscriber);
+ } else {
+ needToHandleFailure = true;
+ }
+ }
+
+ for (ISubscriberRuntime failingSubscriber : failingSubscribers) {
+ try {
+ ingestionRuntime.unsubscribeFeed((CollectionRuntime) failingSubscriber);
+ } catch (Exception e) {
if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to close Feed Runtime Manager " + ioe.getMessage());
+ LOGGER.warning("Excpetion in unsubscribing " + failingSubscriber + " message " + e.getMessage());
}
}
- feedFrameWriter.fail();
+ }
+
+ if (needToHandleFailure) {
+ ingestionRuntime.getAdapterRuntimeManager().setState(State.INACTIVE_INGESTION);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Switching to " + State.INACTIVE_INGESTION + " on occurrence of failure.");
+ }
} else {
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Interrupted Exception, something went wrong");
+ LOGGER.info("Interrupted Exception. None of the subscribers need to handle failures. Shutting down feed ingestion");
}
-
- feedManager.deRegisterFeedRuntime(ingestionRuntime.getFeedRuntimeId());
- feedFrameWriter.close();
+ feedSubscriptionManager.deregisterFeedSubscribableRuntime((SubscribableFeedRuntimeId) ingestionRuntime
+ .getRuntimeId());
throw new HyracksDataException(ie);
}
} catch (Exception e) {
e.printStackTrace();
throw new HyracksDataException(e);
+ } finally {
+ if (ingestionRuntime != null
+ && !ingestionRuntime.getAdapterRuntimeManager().getState().equals(State.INACTIVE_INGESTION)) {
+ feedFrameWriter.close();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Closed Frame Writer " + feedFrameWriter + " adapter state "
+ + ingestionRuntime.getAdapterRuntimeManager().getState());
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Ending intake operator node pushable in state " + State.INACTIVE_INGESTION
+ + " Will resume after correcting failure");
+ }
+ }
+
}
}
- public Map<String, String> getFeedPolicy() {
- return feedPolicy;
+ private void waitTillIngestionIsOver(IAdapterRuntimeManager adapterRuntimeManager) throws InterruptedException {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Waiting for adaptor [" + partition + "]" + "to be done with ingestion of feed " + feedId);
+ }
+ synchronized (adapterRuntimeManager) {
+ while (!(adapterRuntimeManager.getState().equals(IAdapterRuntimeManager.State.FINISHED_INGESTION) || (adapterRuntimeManager
+ .getState().equals(IAdapterRuntimeManager.State.FAILED_INGESTION)))) {
+ adapterRuntimeManager.wait();
+ }
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(" Adaptor " + adapter.getClass().getName() + "[" + partition + "]"
+ + " done with ingestion of feed " + feedId);
+ }
}
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedLifecycleEventSubscriber.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedLifecycleEventSubscriber.java
new file mode 100644
index 0000000..098d713
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedLifecycleEventSubscriber.java
@@ -0,0 +1,62 @@
+/*
+ * 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.Iterator;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.api.IFeedLifecycleEventSubscriber;
+
+public class FeedLifecycleEventSubscriber implements IFeedLifecycleEventSubscriber {
+
+ private LinkedBlockingQueue<FeedLifecycleEvent> inbox;
+
+ public FeedLifecycleEventSubscriber() {
+ this.inbox = new LinkedBlockingQueue<FeedLifecycleEvent>();
+ }
+
+ @Override
+ public void handleFeedEvent(FeedLifecycleEvent event) {
+ inbox.add(event);
+ }
+
+ @Override
+ public void assertEvent(FeedLifecycleEvent event) throws AsterixException, InterruptedException {
+ boolean eventOccurred = false;
+ FeedLifecycleEvent e = null;
+ Iterator<FeedLifecycleEvent> eventsSoFar = inbox.iterator();
+ while (eventsSoFar.hasNext()) {
+ e = eventsSoFar.next();
+ assertNoFailure(e);
+ eventOccurred = e.equals(event);
+ }
+
+ while (!eventOccurred) {
+ e = inbox.take();
+ eventOccurred = e.equals(event);
+ if (!eventOccurred) {
+ assertNoFailure(e);
+ }
+ }
+ }
+
+ private void assertNoFailure(FeedLifecycleEvent e) throws AsterixException {
+ if (e.equals(FeedLifecycleEvent.FEED_INTAKE_FAILURE) || e.equals(FeedLifecycleEvent.FEED_COLLECT_FAILURE)) {
+ throw new AsterixException("Failure in feed");
+ }
+ }
+
+}
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
deleted file mode 100644
index 8b92994..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedManager.java
+++ /dev/null
@@ -1,143 +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.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.common.feeds.FeedConnectionId;
-import edu.uci.ics.asterix.common.feeds.FeedMessageService;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeId;
-import edu.uci.ics.asterix.common.feeds.FeedRuntimeManager;
-import edu.uci.ics.asterix.common.feeds.IFeedManager;
-import edu.uci.ics.asterix.common.feeds.SuperFeedManager;
-
-/**
- * An implementation of the IFeedManager interface.
- * Provider necessary central repository for registering/retrieving
- * artifacts/services associated with a feed.
- */
-public class FeedManager implements IFeedManager {
-
- private static final Logger LOGGER = Logger.getLogger(FeedManager.class.getName());
-
- private Map<FeedConnectionId, FeedRuntimeManager> feedRuntimeManagers = new HashMap<FeedConnectionId, FeedRuntimeManager>();
- private final String nodeId;
-
- public FeedManager(String nodeId) {
- this.nodeId = nodeId;
- }
-
- 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();
- }
-
- @Override
- 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, this);
- 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;
- }
-
- @Override
- public String toString() {
- return "FeedManager " + "[" + nodeId + "]";
- }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
index 9b00322..ca50c83 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMessageOperatorDescriptor.java
@@ -15,6 +15,7 @@
package edu.uci.ics.asterix.metadata.feeds;
import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.api.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;
@@ -29,20 +30,20 @@
private static final long serialVersionUID = 1L;
- private final FeedConnectionId feedId;
+ private final FeedConnectionId connectionId;
private final IFeedMessage feedMessage;
- public FeedMessageOperatorDescriptor(JobSpecification spec, String dataverse, String feedName, String dataset,
+ public FeedMessageOperatorDescriptor(JobSpecification spec, FeedConnectionId connectionId,
IFeedMessage feedMessage) {
super(spec, 0, 1);
- this.feedId = new FeedConnectionId(dataverse, feedName, dataset);
+ this.connectionId = connectionId;
this.feedMessage = feedMessage;
}
@Override
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
- return new FeedMessageOperatorNodePushable(ctx, feedId, feedMessage, partition, nPartitions);
+ return new FeedMessageOperatorNodePushable(ctx, connectionId, 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
index 6380269..09bc5c3 100644
--- 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
@@ -14,40 +14,66 @@
*/
package edu.uci.ics.asterix.metadata.feeds;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
import java.util.logging.Level;
import java.util.logging.Logger;
import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.feeds.CollectionRuntime;
+import edu.uci.ics.asterix.common.feeds.DistributeFeedFrameWriter;
+import edu.uci.ics.asterix.common.feeds.FeedCollectRuntimeInputHandler;
import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedFrameCollector;
+import edu.uci.ics.asterix.common.feeds.FeedFrameCollector.State;
+import edu.uci.ics.asterix.common.feeds.FeedId;
import edu.uci.ics.asterix.common.feeds.FeedRuntime;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeId;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
-import edu.uci.ics.asterix.common.feeds.IFeedManager;
-import edu.uci.ics.asterix.common.feeds.SuperFeedManager;
-import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeInputHandler;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeManager;
+import edu.uci.ics.asterix.common.feeds.FeedTupleCommitResponseMessage;
+import edu.uci.ics.asterix.common.feeds.IngestionRuntime;
+import edu.uci.ics.asterix.common.feeds.IntakePartitionStatistics;
+import edu.uci.ics.asterix.common.feeds.MonitoredBufferTimerTasks.MonitoredBufferStorageTimerTask;
+import edu.uci.ics.asterix.common.feeds.StorageSideMonitoredBuffer;
+import edu.uci.ics.asterix.common.feeds.SubscribableFeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.api.IAdapterRuntimeManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedMessage;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.Mode;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
+import edu.uci.ics.asterix.common.feeds.api.ISubscribableRuntime;
+import edu.uci.ics.asterix.common.feeds.message.EndFeedMessage;
+import edu.uci.ics.asterix.common.feeds.message.ThrottlingEnabledFeedMessage;
+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.base.AbstractUnaryOutputSourceOperatorNodePushable;
/**
- * Runtime for the @see{FeedMessageOperatorDescriptor}
+ * Runtime for the FeedMessageOpertorDescriptor. This operator is responsible for communicating
+ * a feed message to the local feed manager on the host node controller.
+ *
+ * @see FeedMessageOperatorDescriptor
+ * IFeedMessage
+ * IFeedManager
*/
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;
+ private final FeedConnectionId connectionId;
+ private final IFeedMessage message;
private final IFeedManager feedManager;
+ private final int partition;
- public FeedMessageOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId feedId, IFeedMessage feedMessage,
- int partition, int nPartitions) {
- this.feedId = feedId;
- this.feedMessage = feedMessage;
+ public FeedMessageOperatorNodePushable(IHyracksTaskContext ctx, FeedConnectionId connectionId,
+ IFeedMessage feedMessage, int partition, int nPartitions) {
+ this.connectionId = connectionId;
+ this.message = feedMessage;
this.partition = partition;
- this.ctx = ctx;
IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
.getApplicationContext().getApplicationObject();
this.feedManager = runtimeCtx.getFeedManager();
@@ -57,47 +83,38 @@
public void initialize() throws HyracksDataException {
try {
writer.open();
- FeedRuntimeId runtimeId = new FeedRuntimeId(feedId, FeedRuntimeType.INGESTION, partition);
- FeedRuntime feedRuntime = feedManager.getFeedRuntime(runtimeId);
- boolean ingestionLocation = feedRuntime != null;
-
- switch (feedMessage.getMessageType()) {
+ switch (message.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);
- }
+ EndFeedMessage endFeedMessage = (EndFeedMessage) message;
+ switch (endFeedMessage.getEndMessageType()) {
+ case DISCONNECT_FEED:
+ hanldeDisconnectFeedTypeMessage(endFeedMessage);
+ break;
+ case DISCONTINUE_SOURCE:
+ handleDiscontinueFeedTypeMessage(endFeedMessage);
+ break;
}
break;
-
- case SUPER_FEED_MANAGER_ELECT:
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Registering Supers Feed Manager for :" + feedId);
- }
- FeedManagerElectMessage mesg = ((FeedManagerElectMessage) feedMessage);
- SuperFeedManager sfm = new SuperFeedManager(mesg.getFeedId(), mesg.getHost(), mesg.getNodeId(),
- mesg.getPort(), feedManager);
- synchronized (feedManager) {
- INCApplicationContext ncCtx = ctx.getJobletContext().getApplicationContext();
- String nodeId = ncCtx.getNodeId();
- if (sfm.getNodeId().equals(nodeId)) {
- sfm.setLocal(true);
- } else {
- Thread.sleep(5000);
- }
- feedManager.registerSuperFeedManager(feedId, sfm);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Registered super feed mgr " + sfm + " for feed " + feedId);
- }
- }
+ case PREPARE_STALL: {
+ handlePrepareStallMessage((PrepareStallMessage) message);
break;
+ }
+ case TERMINATE_FLOW: {
+ FeedConnectionId connectionId = ((TerminateDataFlowMessage) message).getConnectionId();
+ handleTerminateFlowMessage(connectionId);
+ break;
+ }
+ case COMMIT_ACK_RESPONSE: {
+ handleFeedTupleCommitResponseMessage((FeedTupleCommitResponseMessage) message);
+ break;
+ }
+ case THROTTLING_ENABLED: {
+ handleThrottlingEnabledMessage((ThrottlingEnabledFeedMessage) message);
+ break;
+ }
+ default:
+ break;
+
}
} catch (Exception e) {
@@ -106,4 +123,172 @@
writer.close();
}
}
+
+ private void handleThrottlingEnabledMessage(ThrottlingEnabledFeedMessage throttlingMessage) {
+ FeedConnectionId connectionId = throttlingMessage.getConnectionId();
+ FeedRuntimeManager runtimeManager = feedManager.getFeedConnectionManager().getFeedRuntimeManager(connectionId);
+ Set<FeedRuntimeId> runtimes = runtimeManager.getFeedRuntimes();
+ for (FeedRuntimeId runtimeId : runtimes) {
+ if (runtimeId.getFeedRuntimeType().equals(FeedRuntimeType.STORE)) {
+ FeedRuntime storeRuntime = runtimeManager.getFeedRuntime(runtimeId);
+ ((StorageSideMonitoredBuffer) (storeRuntime.getInputHandler().getmBuffer())).setAcking(false);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Acking Disabled in view of throttling that has been activted upfron in the pipeline "
+ + connectionId);
+ }
+ }
+ }
+ }
+
+ private void handleFeedTupleCommitResponseMessage(FeedTupleCommitResponseMessage commitResponseMessage) {
+ FeedConnectionId connectionId = commitResponseMessage.getConnectionId();
+ FeedRuntimeManager runtimeManager = feedManager.getFeedConnectionManager().getFeedRuntimeManager(connectionId);
+ Set<FeedRuntimeId> runtimes = runtimeManager.getFeedRuntimes();
+ for (FeedRuntimeId runtimeId : runtimes) {
+ FeedRuntime runtime = runtimeManager.getFeedRuntime(runtimeId);
+ switch (runtimeId.getFeedRuntimeType()) {
+ case COLLECT:
+ FeedCollectRuntimeInputHandler inputHandler = (FeedCollectRuntimeInputHandler) runtime
+ .getInputHandler();
+ int maxBasePersisted = commitResponseMessage.getMaxWindowAcked();
+ inputHandler.dropTill(IntakePartitionStatistics.ACK_WINDOW_SIZE * (maxBasePersisted + 1));
+ break;
+ case STORE:
+ MonitoredBufferStorageTimerTask sTask = runtime.getInputHandler().getmBuffer()
+ .getStorageTimeTrackingRateTask();
+ sTask.receiveCommitAckResponse(commitResponseMessage);
+ break;
+ }
+ }
+
+ commitResponseMessage.getIntakePartition();
+ SubscribableFeedRuntimeId sid = new SubscribableFeedRuntimeId(connectionId.getFeedId(), FeedRuntimeType.INTAKE,
+ partition);
+ IngestionRuntime ingestionRuntime = (IngestionRuntime) feedManager.getFeedSubscriptionManager()
+ .getSubscribableRuntime(sid);
+ if (ingestionRuntime != null) {
+ IIntakeProgressTracker tracker = ingestionRuntime.getAdapterRuntimeManager().getProgressTracker();
+ if (tracker != null) {
+ tracker.notifyIngestedTupleTimestamp(System.currentTimeMillis());
+ }
+ }
+ }
+
+ private void handleTerminateFlowMessage(FeedConnectionId connectionId) throws HyracksDataException {
+ FeedRuntimeManager runtimeManager = feedManager.getFeedConnectionManager().getFeedRuntimeManager(connectionId);
+ Set<FeedRuntimeId> feedRuntimes = runtimeManager.getFeedRuntimes();
+
+ boolean found = false;
+ for (FeedRuntimeId runtimeId : feedRuntimes) {
+ FeedRuntime runtime = runtimeManager.getFeedRuntime(runtimeId);
+ if (runtime.getRuntimeId().getRuntimeType().equals(FeedRuntimeType.COLLECT)) {
+ ((CollectionRuntime) runtime).getFrameCollector().setState(State.HANDOVER);
+ found = true;
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Switched " + runtime + " to Hand Over stage");
+ }
+ }
+ }
+ if (!found) {
+ throw new HyracksDataException("COLLECT Runtime not found!");
+ }
+ }
+
+ private void handlePrepareStallMessage(PrepareStallMessage prepareStallMessage) throws HyracksDataException {
+ FeedConnectionId connectionId = prepareStallMessage.getConnectionId();
+ int computePartitionsRetainLimit = prepareStallMessage.getComputePartitionsRetainLimit();
+ FeedRuntimeManager runtimeManager = feedManager.getFeedConnectionManager().getFeedRuntimeManager(connectionId);
+ Set<FeedRuntimeId> feedRuntimes = runtimeManager.getFeedRuntimes();
+ for (FeedRuntimeId runtimeId : feedRuntimes) {
+ FeedRuntime runtime = runtimeManager.getFeedRuntime(runtimeId);
+ switch (runtimeId.getFeedRuntimeType()) {
+ case COMPUTE:
+ Mode requiredMode = runtimeId.getPartition() <= computePartitionsRetainLimit ? Mode.STALL
+ : Mode.END;
+ runtime.setMode(requiredMode);
+ break;
+ default:
+ runtime.setMode(Mode.STALL);
+ break;
+ }
+ }
+ }
+
+ private void handleDiscontinueFeedTypeMessage(EndFeedMessage endFeedMessage) throws Exception {
+ FeedId sourceFeedId = endFeedMessage.getSourceFeedId();
+ SubscribableFeedRuntimeId subscribableRuntimeId = new SubscribableFeedRuntimeId(sourceFeedId,
+ FeedRuntimeType.INTAKE, partition);
+ ISubscribableRuntime feedRuntime = feedManager.getFeedSubscriptionManager().getSubscribableRuntime(
+ subscribableRuntimeId);
+ IAdapterRuntimeManager adapterRuntimeManager = ((IngestionRuntime) feedRuntime).getAdapterRuntimeManager();
+ adapterRuntimeManager.stop();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Stopped Adapter " + adapterRuntimeManager);
+ }
+ }
+
+ private void hanldeDisconnectFeedTypeMessage(EndFeedMessage endFeedMessage) throws Exception {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Ending feed:" + endFeedMessage.getFeedConnectionId());
+ }
+ FeedRuntimeId runtimeId = null;
+ FeedRuntimeType subscribableRuntimeType = ((EndFeedMessage) message).getSourceRuntimeType();
+ if (endFeedMessage.isCompleteDisconnection()) {
+ // subscribableRuntimeType represents the location at which the feed connection receives data
+ FeedRuntimeType runtimeType = null;
+ switch (subscribableRuntimeType) {
+ case INTAKE:
+ runtimeType = FeedRuntimeType.COLLECT;
+ break;
+ case COMPUTE:
+ runtimeType = FeedRuntimeType.COMPUTE_COLLECT;
+ break;
+ default:
+ throw new IllegalStateException("Invalid subscribable runtime type " + subscribableRuntimeType);
+ }
+
+ runtimeId = new FeedRuntimeId(runtimeType, partition, FeedRuntimeId.DEFAULT_OPERAND_ID);
+ CollectionRuntime feedRuntime = (CollectionRuntime) feedManager.getFeedConnectionManager().getFeedRuntime(
+ connectionId, runtimeId);
+ feedRuntime.getSourceRuntime().unsubscribeFeed(feedRuntime);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Complete Unsubscription of " + endFeedMessage.getFeedConnectionId());
+ }
+ } else {
+ // subscribaleRuntimeType represents the location for data hand-off in presence of subscribers
+ switch (subscribableRuntimeType) {
+ case INTAKE:
+ // illegal state as data hand-off from one feed to another does not happen at intake
+ throw new IllegalStateException("Illegal State, invalid runtime type " + subscribableRuntimeType);
+ case COMPUTE:
+ // feed could be primary or secondary, doesn't matter
+ SubscribableFeedRuntimeId feedSubscribableRuntimeId = new SubscribableFeedRuntimeId(
+ connectionId.getFeedId(), FeedRuntimeType.COMPUTE, partition);
+ ISubscribableRuntime feedRuntime = feedManager.getFeedSubscriptionManager().getSubscribableRuntime(
+ feedSubscribableRuntimeId);
+ DistributeFeedFrameWriter dWriter = (DistributeFeedFrameWriter) feedRuntime.getFeedFrameWriter();
+ Map<IFrameWriter, FeedFrameCollector> registeredCollectors = dWriter.getRegisteredReaders();
+
+ IFrameWriter unsubscribingWriter = null;
+ for (Entry<IFrameWriter, FeedFrameCollector> entry : registeredCollectors.entrySet()) {
+ IFrameWriter frameWriter = entry.getKey();
+ FeedRuntimeInputHandler feedFrameWriter = (FeedRuntimeInputHandler) frameWriter;
+ if (feedFrameWriter.getConnectionId().equals(endFeedMessage.getFeedConnectionId())) {
+ unsubscribingWriter = feedFrameWriter;
+ dWriter.unsubscribeFeed(unsubscribingWriter);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Partial Unsubscription of " + unsubscribingWriter);
+ }
+ break;
+ }
+ }
+ break;
+ }
+
+ }
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Unsubscribed from feed :" + connectionId);
+ }
+ }
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaComputeNodePushable.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaComputeNodePushable.java
new file mode 100644
index 0000000..371e5b6
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaComputeNodePushable.java
@@ -0,0 +1,207 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.feeds.DistributeFeedFrameWriter;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeInputHandler;
+import edu.uci.ics.asterix.common.feeds.SubscribableFeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.SubscribableRuntime;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.Mode;
+import edu.uci.ics.asterix.common.feeds.api.ISubscribableRuntime;
+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.value.IRecordDescriptorProvider;
+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.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class FeedMetaComputeNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedMetaComputeNodePushable.class.getName());
+
+ /** Runtime node pushable corresponding to the core feed operator **/
+ private AbstractUnaryInputUnaryOutputOperatorNodePushable coreOperator;
+
+ /**
+ * A policy enforcer that ensures dynamic 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 connectionId;
+
+ /**
+ * Denotes the i'th operator instance in a setting where K operator
+ * instances are scheduled to run in parallel
+ **/
+ private int partition;
+
+ private int nPartitions;
+
+ /** The (singleton) instance of IFeedManager **/
+ private IFeedManager feedManager;
+
+ private FrameTupleAccessor fta;
+
+ private final IHyracksTaskContext ctx;
+
+ private final FeedRuntimeType runtimeType = FeedRuntimeType.COMPUTE;
+
+ private FeedRuntimeInputHandler inputSideHandler;
+
+ public FeedMetaComputeNodePushable(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
+ int partition, int nPartitions, IOperatorDescriptor coreOperator, FeedConnectionId feedConnectionId,
+ Map<String, String> feedPolicyProperties, String operationId) throws HyracksDataException {
+ this.ctx = ctx;
+ this.coreOperator = (AbstractUnaryInputUnaryOutputOperatorNodePushable) ((IActivity) coreOperator)
+ .createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+ this.policyEnforcer = new FeedPolicyEnforcer(feedConnectionId, feedPolicyProperties);
+ this.partition = partition;
+ this.nPartitions = nPartitions;
+ this.connectionId = feedConnectionId;
+ this.feedManager = ((IAsterixAppRuntimeContext) (IAsterixAppRuntimeContext) ctx.getJobletContext()
+ .getApplicationContext().getApplicationObject()).getFeedManager();
+ IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+ .getApplicationContext().getApplicationObject();
+ this.feedManager = runtimeCtx.getFeedManager();
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ FeedRuntimeId runtimeId = new SubscribableFeedRuntimeId(connectionId.getFeedId(), runtimeType, partition);
+ try {
+ feedRuntime = feedManager.getFeedConnectionManager().getFeedRuntime(connectionId, runtimeId);
+ if (feedRuntime == null) {
+ initializeNewFeedRuntime(runtimeId);
+ } else {
+ reviveOldFeedRuntime(runtimeId);
+ }
+ writer.open();
+ coreOperator.open();
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ private void initializeNewFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
+ this.fta = new FrameTupleAccessor(recordDesc);
+ this.inputSideHandler = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, coreOperator,
+ policyEnforcer.getFeedPolicyAccessor(), true, fta, recordDesc, feedManager,
+ nPartitions);
+
+ DistributeFeedFrameWriter distributeWriter = new DistributeFeedFrameWriter(ctx, connectionId.getFeedId(), writer,
+ runtimeType, partition, new FrameTupleAccessor(recordDesc), feedManager);
+ coreOperator.setOutputFrameWriter(0, distributeWriter, recordDesc);
+
+ feedRuntime = new SubscribableRuntime(connectionId.getFeedId(), runtimeId, inputSideHandler, distributeWriter,
+ recordDesc);
+ feedManager.getFeedSubscriptionManager().registerFeedSubscribableRuntime((ISubscribableRuntime) feedRuntime);
+ feedManager.getFeedConnectionManager().registerFeedRuntime(connectionId, feedRuntime);
+
+ distributeWriter.subscribeFeed(policyEnforcer.getFeedPolicyAccessor(), writer, connectionId);
+ }
+
+ private void reviveOldFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
+ this.fta = new FrameTupleAccessor(recordDesc);
+ this.inputSideHandler = feedRuntime.getInputHandler();
+ this.inputSideHandler.setCoreOperator(coreOperator);
+
+ DistributeFeedFrameWriter distributeWriter = new DistributeFeedFrameWriter(ctx, connectionId.getFeedId(), writer,
+ runtimeType, partition, new FrameTupleAccessor(recordDesc), feedManager);
+ coreOperator.setOutputFrameWriter(0, distributeWriter, recordDesc);
+ distributeWriter.subscribeFeed(policyEnforcer.getFeedPolicyAccessor(), writer, connectionId);
+
+ inputSideHandler.reset(nPartitions);
+ feedRuntime.setMode(Mode.PROCESS);
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ try {
+ inputSideHandler.nextFrame(buffer);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Core Op:" + coreOperator.getDisplayName() + " fail ");
+ }
+ feedRuntime.setMode(Mode.FAIL);
+ coreOperator.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ boolean stalled = inputSideHandler.getMode().equals(Mode.STALL);
+ boolean end = inputSideHandler.getMode().equals(Mode.END);
+ try {
+ if (inputSideHandler != null) {
+ if (!(stalled || end)) {
+ inputSideHandler.nextFrame(null); // signal end of data
+ while (!inputSideHandler.isFinished()) {
+ synchronized (coreOperator) {
+ coreOperator.wait();
+ }
+ }
+ } else {
+ inputSideHandler.setFinished(true);
+ }
+ }
+ coreOperator.close();
+ System.out.println("CLOSED " + coreOperator + " STALLED ?" + stalled + " ENDED " + end);
+ } catch (Exception e) {
+ e.printStackTrace();
+ } finally {
+ if (!stalled) {
+ deregister();
+ System.out.println("DEREGISTERING " + this.feedRuntime.getRuntimeId());
+ } else {
+ System.out.println("NOT DEREGISTERING " + this.feedRuntime.getRuntimeId());
+ }
+ if (inputSideHandler != null) {
+ inputSideHandler.close();
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Ending Operator " + this.feedRuntime.getRuntimeId());
+ }
+ }
+ }
+
+ private void deregister() {
+ if (feedRuntime != null) {
+ // deregister from subscription manager
+ SubscribableFeedRuntimeId runtimeId = (SubscribableFeedRuntimeId) feedRuntime.getRuntimeId();
+ feedManager.getFeedSubscriptionManager().deregisterFeedSubscribableRuntime(runtimeId);
+
+ // deregister from connection manager
+ feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId,
+ ((FeedRuntime) feedRuntime).getRuntimeId());
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaNodePushable.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaNodePushable.java
new file mode 100644
index 0000000..d348cfc
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaNodePushable.java
@@ -0,0 +1,170 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeInputHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.Mode;
+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.value.IRecordDescriptorProvider;
+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.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class FeedMetaNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedMetaNodePushable.class.getName());
+
+ /** Runtime node pushable corresponding to the core feed operator **/
+ private AbstractUnaryInputUnaryOutputOperatorNodePushable coreOperator;
+
+ /**
+ * 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 connectionId;
+
+ /**
+ * Denotes the i'th operator instance in a setting where K operator
+ * instances are scheduled to run in parallel
+ **/
+ private int partition;
+
+ /** Total number of partitions available **/
+ private int nPartitions;
+
+ /** Type associated with the core feed operator **/
+ private final FeedRuntimeType runtimeType = FeedRuntimeType.OTHER;
+
+ /** The (singleton) instance of IFeedManager **/
+ private IFeedManager feedManager;
+
+ private FrameTupleAccessor fta;
+
+ private final IHyracksTaskContext ctx;
+
+ private final String operandId;
+
+ /** The pre-processor associated with this runtime **/
+ private FeedRuntimeInputHandler inputSideHandler;
+
+ public FeedMetaNodePushable(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider, int partition,
+ int nPartitions, IOperatorDescriptor coreOperator, FeedConnectionId feedConnectionId,
+ Map<String, String> feedPolicyProperties, String operationId) throws HyracksDataException {
+ this.ctx = ctx;
+ this.coreOperator = (AbstractUnaryInputUnaryOutputOperatorNodePushable) ((IActivity) coreOperator)
+ .createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+ this.policyEnforcer = new FeedPolicyEnforcer(feedConnectionId, feedPolicyProperties);
+ this.partition = partition;
+ this.nPartitions = nPartitions;
+ this.connectionId = feedConnectionId;
+ this.feedManager = ((IAsterixAppRuntimeContext) (IAsterixAppRuntimeContext) ctx.getJobletContext()
+ .getApplicationContext().getApplicationObject()).getFeedManager();
+ IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+ .getApplicationContext().getApplicationObject();
+ this.feedManager = runtimeCtx.getFeedManager();
+ this.operandId = operationId;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ FeedRuntimeId runtimeId = new FeedRuntimeId(runtimeType, partition, operandId);
+ try {
+ feedRuntime = feedManager.getFeedConnectionManager().getFeedRuntime(connectionId, runtimeId);
+ if (feedRuntime == null) {
+ initializeNewFeedRuntime(runtimeId);
+ } else {
+ reviveOldFeedRuntime(runtimeId);
+ }
+ coreOperator.open();
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ private void initializeNewFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
+ this.fta = new FrameTupleAccessor(recordDesc);
+ this.inputSideHandler = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId,
+ (AbstractUnaryInputUnaryOutputOperatorNodePushable) coreOperator,
+ policyEnforcer.getFeedPolicyAccessor(), false, fta, recordDesc, feedManager,
+ nPartitions);
+
+ setupBasicRuntime(inputSideHandler);
+ }
+
+ private void reviveOldFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
+ this.inputSideHandler = feedRuntime.getInputHandler();
+ this.fta = new FrameTupleAccessor(recordDesc);
+ coreOperator.setOutputFrameWriter(0, writer, recordDesc);
+ feedRuntime.setMode(Mode.PROCESS);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Retreived state from the zombie instance " + runtimeType + " node.");
+ }
+ }
+
+ private void setupBasicRuntime(FeedRuntimeInputHandler inputHandler) throws Exception {
+ coreOperator.setOutputFrameWriter(0, writer, recordDesc);
+ FeedRuntimeId runtimeId = new FeedRuntimeId(runtimeType, partition, operandId);
+ feedRuntime = new FeedRuntime(runtimeId, inputHandler, writer);
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ try {
+ inputSideHandler.nextFrame(buffer);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.info("Core Op:" + coreOperator.getDisplayName() + " fail ");
+ }
+ feedRuntime.setMode(Mode.FAIL);
+ coreOperator.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ coreOperator.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ // ignore
+ } finally {
+ if (inputSideHandler != null) {
+ inputSideHandler.close();
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Ending Operator " + this.feedRuntime.getRuntimeId());
+ }
+ }
+ }
+
+}
\ No newline at end of file
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
index d636e04..953853d 100644
--- 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
@@ -1,75 +1,80 @@
+/*
+ * 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.nio.ByteBuffer;
-import java.util.logging.Level;
-import java.util.logging.Logger;
+import java.util.Map;
-import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeId;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeState;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
-import edu.uci.ics.asterix.common.feeds.IFeedManager;
-import edu.uci.ics.asterix.metadata.entities.FeedPolicy;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
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.
+ * 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 Adapter **/
+ /**
+ * The actual (Hyracks) operator that is wrapped around by the MetaFeed
+ * operator.
+ **/
private IOperatorDescriptor coreOperator;
/**
- * A unique identifier for the feed instance. A feed instance represents the flow of data
- * from a feed to a dataset.
+ * 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;
+ **/
+ private final Map<String, String> feedPolicyProperties;
/**
* type for the feed runtime associated with the operator.
- * Possible values: INGESTION, COMPUTE, STORAGE, COMMIT
- */
+ * Possible values: COMPUTE, STORE, OTHER
+ **/
private final FeedRuntimeType runtimeType;
private final String operandId;
public FeedMetaOperatorDescriptor(JobSpecification spec, FeedConnectionId feedConnectionId,
- IOperatorDescriptor coreOperatorDescriptor, FeedPolicy feedPolicy, FeedRuntimeType runtimeType,
- String operandId) {
+ IOperatorDescriptor coreOperatorDescriptor, Map<String, String> feedPolicyProperties,
+ FeedRuntimeType runtimeType, boolean enableSubscriptionMode, String operandId) {
super(spec, coreOperatorDescriptor.getInputArity(), coreOperatorDescriptor.getOutputArity());
this.feedConnectionId = feedConnectionId;
- this.feedPolicy = feedPolicy;
+ this.feedPolicyProperties = feedPolicyProperties;
if (coreOperatorDescriptor.getOutputRecordDescriptors().length == 1) {
recordDescriptors[0] = coreOperatorDescriptor.getOutputRecordDescriptors()[0];
}
@@ -81,8 +86,30 @@
@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, operandId);
+ IOperatorNodePushable nodePushable = null;
+ switch (runtimeType) {
+ case COMPUTE:
+ nodePushable = new FeedMetaComputeNodePushable(ctx, recordDescProvider, partition, nPartitions,
+ coreOperator, feedConnectionId, feedPolicyProperties, operandId);
+ break;
+ case STORE:
+ nodePushable = new FeedMetaStoreNodePushable(ctx, recordDescProvider, partition, nPartitions,
+ coreOperator, feedConnectionId, feedPolicyProperties, operandId);
+ break;
+ case OTHER:
+ nodePushable = new FeedMetaNodePushable(ctx, recordDescProvider, partition, nPartitions, coreOperator,
+ feedConnectionId, feedPolicyProperties, operandId);
+ break;
+ case ETS:
+ nodePushable = ((AlgebricksMetaOperatorDescriptor) coreOperator).createPushRuntime(ctx,
+ recordDescProvider, partition, nPartitions);
+ break;
+ case JOIN:
+ break;
+ default:
+ throw new HyracksDataException(new IllegalArgumentException("Invalid feed runtime: " + runtimeType));
+ }
+ return nodePushable;
}
@Override
@@ -90,173 +117,12 @@
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;
-
- /** The (singleton) instance of IFeedManager **/
- private IFeedManager feedManager;
-
- private final String operandId;
-
- public FeedMetaNodePushable(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
- int partition, int nPartitions, IOperatorDescriptor coreOperator, FeedConnectionId feedConnectionId,
- FeedPolicy feedPolicy, FeedRuntimeType runtimeType, String operationId) 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(recordDesc);
- IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
- .getApplicationContext().getApplicationObject();
- this.feedManager = runtimeCtx.getFeedManager();
- this.operandId = operationId;
- }
-
- @Override
- public void open() throws HyracksDataException {
- FeedRuntimeId runtimeId = new FeedRuntimeId(feedId, runtimeType, operandId, partition);
- try {
- feedRuntime = feedManager.getFeedRuntime(runtimeId);
- if (feedRuntime == null) {
- feedRuntime = new FeedRuntime(feedId, partition, runtimeType, operandId);
- feedManager.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, feedManager);
- 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) {
- FeedRuntimeState runtimeState = feedRuntime.getRuntimeState();
- if (runtimeState != null) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("State from previous zombie instance " + feedRuntime.getRuntimeState());
- }
- coreOperatorNodePushable.nextFrame(feedRuntime.getRuntimeState().getFrame());
- feedRuntime.setRuntimeState(null);
- }
- resumeOldState = false;
- }
- currentBuffer = buffer;
- coreOperatorNodePushable.nextFrame(buffer);
- currentBuffer = null;
- } catch (HyracksDataException e) {
- if (policyEnforcer.getFeedPolicyAccessor().continueOnApplicationFailure()) {
- boolean isExceptionHarmful = e.getCause() instanceof TreeIndexException && !resumeOldState;
- if (isExceptionHarmful) {
- // TODO: log the tuple
- FeedRuntimeState runtimeState = new FeedRuntimeState(buffer, writer, e);
- feedRuntime.setRuntimeState(runtimeState);
- } 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;
- }
- }
- }
-
- @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.deRegisterFeedRuntime(feedRuntime.getFeedRuntimeId());
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("No state to save, de-registered feed runtime " + feedRuntime.getFeedRuntimeId());
- }
- }
- }
- coreOperatorNodePushable.fail();
- }
-
- @Override
- public void close() throws HyracksDataException {
- coreOperatorNodePushable.close();
- feedManager.deRegisterFeedRuntime(feedRuntime.getFeedRuntimeId());
- }
-
- }
-
public IOperatorDescriptor getCoreOperator() {
return coreOperator;
}
+ public FeedRuntimeType getRuntimeType() {
+ return runtimeType;
+ }
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaStoreNodePushable.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaStoreNodePushable.java
new file mode 100644
index 0000000..1bb377c
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedMetaStoreNodePushable.java
@@ -0,0 +1,198 @@
+package edu.uci.ics.asterix.metadata.feeds;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntime;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeInputHandler;
+import edu.uci.ics.asterix.common.feeds.api.IFeedManager;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.Mode;
+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.value.IRecordDescriptorProvider;
+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.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class FeedMetaStoreNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+
+ private static final Logger LOGGER = Logger.getLogger(FeedMetaStoreNodePushable.class.getName());
+
+ /** Runtime node pushable corresponding to the core feed operator **/
+ private AbstractUnaryInputUnaryOutputOperatorNodePushable coreOperator;
+
+ /**
+ * 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 connectionId;
+
+ /**
+ * Denotes the i'th operator instance in a setting where K operator
+ * instances are scheduled to run in parallel
+ **/
+ private int partition;
+
+ private int nPartitions;
+
+ /** Type associated with the core feed operator **/
+ private final FeedRuntimeType runtimeType = FeedRuntimeType.STORE;
+
+ /** The (singleton) instance of IFeedManager **/
+ private IFeedManager feedManager;
+
+ private FrameTupleAccessor fta;
+
+ private final IHyracksTaskContext ctx;
+
+ private final String operandId;
+
+ private FeedRuntimeInputHandler inputSideHandler;
+
+ public FeedMetaStoreNodePushable(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
+ int partition, int nPartitions, IOperatorDescriptor coreOperator, FeedConnectionId feedConnectionId,
+ Map<String, String> feedPolicyProperties, String operationId) throws HyracksDataException {
+ this.ctx = ctx;
+ this.coreOperator = (AbstractUnaryInputUnaryOutputOperatorNodePushable) ((IActivity) coreOperator)
+ .createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+ this.policyEnforcer = new FeedPolicyEnforcer(feedConnectionId, feedPolicyProperties);
+ this.partition = partition;
+ this.nPartitions = nPartitions;
+ this.connectionId = feedConnectionId;
+ this.feedManager = ((IAsterixAppRuntimeContext) (IAsterixAppRuntimeContext) ctx.getJobletContext()
+ .getApplicationContext().getApplicationObject()).getFeedManager();
+ IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+ .getApplicationContext().getApplicationObject();
+ this.feedManager = runtimeCtx.getFeedManager();
+ this.operandId = operationId;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ FeedRuntimeId runtimeId = new FeedRuntimeId(runtimeType, partition, operandId);
+ try {
+ feedRuntime = feedManager.getFeedConnectionManager().getFeedRuntime(connectionId, runtimeId);
+ if (feedRuntime == null) {
+ initializeNewFeedRuntime(runtimeId);
+ } else {
+ reviveOldFeedRuntime(runtimeId);
+ }
+
+ coreOperator.open();
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ private void initializeNewFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Runtime not found for " + runtimeId + " connection id " + connectionId);
+ }
+ this.fta = new FrameTupleAccessor(recordDesc);
+ this.inputSideHandler = new FeedRuntimeInputHandler(ctx, connectionId, runtimeId, coreOperator,
+ policyEnforcer.getFeedPolicyAccessor(), true, fta, recordDesc, feedManager,
+ nPartitions);
+ setupBasicRuntime(inputSideHandler);
+ }
+
+ private void reviveOldFeedRuntime(FeedRuntimeId runtimeId) throws Exception {
+ this.inputSideHandler = feedRuntime.getInputHandler();
+ this.fta = new FrameTupleAccessor(recordDesc);
+ coreOperator.setOutputFrameWriter(0, writer, recordDesc);
+ this.inputSideHandler.reset(nPartitions);
+ this.inputSideHandler.setCoreOperator(coreOperator);
+ feedRuntime.setMode(Mode.PROCESS);
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Retreived state from the zombie instance from previous execution for " + runtimeType
+ + " node.");
+ }
+ }
+
+ private void setupBasicRuntime(FeedRuntimeInputHandler inputHandler) throws Exception {
+ coreOperator.setOutputFrameWriter(0, writer, recordDesc);
+ FeedRuntimeId runtimeId = new FeedRuntimeId(runtimeType, partition, operandId);
+ feedRuntime = new FeedRuntime(runtimeId, inputHandler, writer);
+ feedManager.getFeedConnectionManager().registerFeedRuntime(connectionId, (FeedRuntime) feedRuntime);
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ try {
+ inputSideHandler.nextFrame(buffer);
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.info("Core Op:" + coreOperator.getDisplayName() + " fail ");
+ }
+ feedRuntime.setMode(Mode.FAIL);
+ coreOperator.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ System.out.println("CLOSE CALLED FOR " + this.feedRuntime.getRuntimeId());
+ boolean stalled = inputSideHandler.getMode().equals(Mode.STALL);
+ try {
+ if (!stalled) {
+ System.out.println("SIGNALLING END OF DATA for " + this.feedRuntime.getRuntimeId() + " mode is "
+ + inputSideHandler.getMode() + " WAITING ON " + coreOperator);
+ inputSideHandler.nextFrame(null); // signal end of data
+ while (!inputSideHandler.isFinished()) {
+ synchronized (coreOperator) {
+ coreOperator.wait();
+ }
+ }
+ System.out.println("ABOUT TO CLOSE OPERATOR " + coreOperator);
+ }
+ coreOperator.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ // ignore
+ } finally {
+ if (!stalled) {
+ deregister();
+ System.out.println("DEREGISTERING " + this.feedRuntime.getRuntimeId());
+ } else {
+ System.out.println("NOT DEREGISTERING " + this.feedRuntime.getRuntimeId());
+ }
+ inputSideHandler.close();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Ending Operator " + this.feedRuntime.getRuntimeId());
+ }
+ }
+ }
+
+ private void deregister() {
+ if (feedRuntime != null) {
+ feedManager.getFeedConnectionManager().deRegisterFeedRuntime(connectionId,
+ ((FeedRuntime) feedRuntime).getRuntimeId());
+ }
+ }
+
+}
\ No newline at end of file
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
deleted file mode 100644
index fd9716c..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedPolicyAccessor.java
+++ /dev/null
@@ -1,105 +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.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
index 44487ec..5cd4bf1 100644
--- 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
@@ -15,60 +15,32 @@
package edu.uci.ics.asterix.metadata.feeds;
import java.rmi.RemoteException;
-import java.util.HashMap;
import java.util.Map;
import edu.uci.ics.asterix.common.exceptions.ACIDException;
import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
-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;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
public class FeedPolicyEnforcer {
- private final FeedConnectionId feedId;
- private final FeedPolicyAccessor feedPolicyAccessor;
- private final FeedActivity feedActivity;
+ private final FeedConnectionId connectionId;
+ private final FeedPolicyAccessor policyAccessor;
- 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 FeedPolicyEnforcer(FeedConnectionId feedConnectionId, Map<String, String> feedPolicy) {
+ this.connectionId = feedConnectionId;
+ this.policyAccessor = new FeedPolicyAccessor(feedPolicy);
}
public boolean continueIngestionPostSoftwareFailure(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();
- }
+ return policyAccessor.continueOnSoftFailure();
}
public FeedPolicyAccessor getFeedPolicyAccessor() {
- return feedPolicyAccessor;
+ return policyAccessor;
}
public FeedConnectionId getFeedId() {
- return feedId;
+ return connectionId;
}
}
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
deleted file mode 100644
index d3225a2..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedReport.java
+++ /dev/null
@@ -1,117 +0,0 @@
-package edu.uci.ics.asterix.metadata.feeds;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
-import edu.uci.ics.asterix.common.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/FeedSubscriptionManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedSubscriptionManager.java
new file mode 100644
index 0000000..48102ba
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedSubscriptionManager.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.metadata.feeds;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.feeds.SubscribableFeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedSubscriptionManager;
+import edu.uci.ics.asterix.common.feeds.api.ISubscribableRuntime;
+
+public class FeedSubscriptionManager implements IFeedSubscriptionManager {
+
+ private static Logger LOGGER = Logger.getLogger(FeedSubscriptionManager.class.getName());
+
+ private final String nodeId;
+
+ private final Map<SubscribableFeedRuntimeId, ISubscribableRuntime> subscribableRuntimes;
+
+ public FeedSubscriptionManager(String nodeId) {
+ this.nodeId = nodeId;
+ this.subscribableRuntimes = new HashMap<SubscribableFeedRuntimeId, ISubscribableRuntime>();
+ }
+
+ @Override
+ public void registerFeedSubscribableRuntime(ISubscribableRuntime subscribableRuntime) {
+ SubscribableFeedRuntimeId sid = (SubscribableFeedRuntimeId) subscribableRuntime.getRuntimeId();
+ if (!subscribableRuntimes.containsKey(sid)) {
+ subscribableRuntimes.put(sid, subscribableRuntime);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Registered feed subscribable runtime " + subscribableRuntime);
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Feed ingestion runtime " + subscribableRuntime + " already registered.");
+ }
+ }
+ }
+
+ @Override
+ public ISubscribableRuntime getSubscribableRuntime(SubscribableFeedRuntimeId subscribableFeedRuntimeId) {
+ return subscribableRuntimes.get(subscribableFeedRuntimeId);
+ }
+
+ @Override
+ public void deregisterFeedSubscribableRuntime(SubscribableFeedRuntimeId ingestionId) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("De-registered feed subscribable runtime " + ingestionId);
+ }
+ subscribableRuntimes.remove(ingestionId);
+ }
+
+ @Override
+ public String toString() {
+ return "IngestionManager [" + nodeId + "]";
+ }
+
+}
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
index c7b905d..ebee738 100644
--- 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
@@ -21,35 +21,47 @@
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
+import java.util.Set;
import java.util.logging.Level;
import java.util.logging.Logger;
import org.apache.commons.lang3.tuple.Pair;
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
import edu.uci.ics.asterix.common.dataflow.AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor;
import edu.uci.ics.asterix.common.dataflow.AsterixLSMTreeInsertDeleteOperatorDescriptor;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime.FeedRuntimeType;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.feeds.FeedRuntimeId;
+import edu.uci.ics.asterix.common.feeds.api.IFeedRuntime.FeedRuntimeType;
+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.bootstrap.MetadataConstants;
import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter.AdapterType;
import edu.uci.ics.asterix.metadata.entities.Datatype;
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.PrimaryFeed;
+import edu.uci.ics.asterix.metadata.entities.SecondaryFeed;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory;
import edu.uci.ics.asterix.metadata.functions.ExternalLibraryManager;
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;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
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.algebricks.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
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;
@@ -61,7 +73,11 @@
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.dataflow.value.ITuplePartitionComputerFactory;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.RandomPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
/**
* A utility class for providing helper functions for feeds
@@ -70,9 +86,17 @@
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 String getFeedPointKeyRep(Feed feed, List<String> appliedFunctions) {
+ StringBuilder builder = new StringBuilder();
+ builder.append(feed.getDataverseName() + ":");
+ builder.append(feed.getFeedName() + ":");
+ if (appliedFunctions != null && !appliedFunctions.isEmpty()) {
+ for (String function : appliedFunctions) {
+ builder.append(function + ":");
+ }
+ builder.deleteCharAt(builder.length() - 1);
+ }
+ return builder.toString();
}
private static class LocationConstraint {
@@ -80,62 +104,113 @@
String location;
}
- public static JobSpecification alterJobSpecificationForFeed(JobSpecification spec,
- FeedConnectionId feedConnectionId, FeedPolicy feedPolicy) {
+ public static Dataset validateIfDatasetExists(String dataverse, String datasetName, MetadataTransactionContext ctx)
+ throws AsterixException {
+ Dataset dataset = MetadataManager.INSTANCE.getDataset(ctx, dataverse, datasetName);
+ if (dataset == null) {
+ throw new AsterixException("Unknown target dataset :" + datasetName);
+ }
- FeedPolicyAccessor fpa = new FeedPolicyAccessor(feedPolicy.getProperties());
- boolean alterationRequired = (fpa.collectStatistics() || fpa.continueOnApplicationFailure()
- || fpa.continueOnHardwareFailure() || fpa.isElastic());
- if (!alterationRequired) {
- return spec;
+ if (!dataset.getDatasetType().equals(DatasetType.INTERNAL)) {
+ throw new AsterixException("Statement not applicable. Dataset " + datasetName + " is not of required type "
+ + DatasetType.INTERNAL);
+ }
+ return dataset;
+ }
+
+ public static Feed validateIfFeedExists(String dataverse, String feedName, MetadataTransactionContext ctx)
+ throws MetadataException, AsterixException {
+ Feed feed = MetadataManager.INSTANCE.getFeed(ctx, dataverse, feedName);
+ if (feed == null) {
+ throw new AsterixException("Unknown source feed: " + feedName);
+ }
+ return feed;
+ }
+
+ public static FeedPolicy validateIfPolicyExists(String dataverse, String policyName, MetadataTransactionContext ctx)
+ throws AsterixException {
+ FeedPolicy feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(ctx, dataverse, policyName);
+ if (feedPolicy == null) {
+ feedPolicy = MetadataManager.INSTANCE.getFeedPolicy(ctx, MetadataConstants.METADATA_DATAVERSE_NAME,
+ policyName);
+ if (feedPolicy == null) {
+ throw new AsterixException("Unknown feed policy" + policyName);
+ }
+ }
+ return feedPolicy;
+ }
+
+ public static JobSpecification alterJobSpecificationForFeed(JobSpecification spec,
+ FeedConnectionId feedConnectionId, Map<String, String> feedPolicyProperties) {
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Original Job Spec:" + spec);
}
JobSpecification altered = new JobSpecification(spec.getFrameSize());
Map<OperatorDescriptorId, IOperatorDescriptor> operatorMap = spec.getOperatorMap();
-
+ boolean preProcessingRequired = preProcessingRequired(feedConnectionId);
// copy operators
- String operationId = null;
+ String operandId = null;
Map<OperatorDescriptorId, OperatorDescriptorId> oldNewOID = new HashMap<OperatorDescriptorId, OperatorDescriptorId>();
+ FeedMetaOperatorDescriptor metaOp = null;
for (Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operatorMap.entrySet()) {
- operationId = FeedRuntime.FeedRuntimeId.DEFAULT_OPERATION_ID;
+ operandId = FeedRuntimeId.DEFAULT_OPERAND_ID;
IOperatorDescriptor opDesc = entry.getValue();
- if (opDesc instanceof FeedIntakeOperatorDescriptor) {
- FeedIntakeOperatorDescriptor orig = (FeedIntakeOperatorDescriptor) opDesc;
- FeedIntakeOperatorDescriptor fiop;
- if (orig.getAdapterFactory() != null) {
- fiop = new FeedIntakeOperatorDescriptor(altered, orig.getFeedId(), orig.getAdapterFactory(),
- (ARecordType) orig.getOutputType(), orig.getRecordDescriptor(), orig.getFeedPolicy());
- } else {
- fiop = new FeedIntakeOperatorDescriptor(altered, orig.getFeedId(), orig.getAdapterLibraryName(),
- orig.getAdapterFactoryClassName(), orig.getAdapterConfiguration(),
- (ARecordType) orig.getOutputType(), orig.getRecordDescriptor(), orig.getFeedPolicy());
- }
+ if (opDesc instanceof FeedCollectOperatorDescriptor) {
+ FeedCollectOperatorDescriptor orig = (FeedCollectOperatorDescriptor) opDesc;
+ FeedCollectOperatorDescriptor fiop = new FeedCollectOperatorDescriptor(altered,
+ orig.getFeedConnectionId(), orig.getSourceFeedId(), (ARecordType) orig.getOutputType(),
+ orig.getRecordDescriptor(), orig.getFeedPolicyProperties(), orig.getSubscriptionLocation());
oldNewOID.put(opDesc.getOperatorId(), fiop.getOperatorId());
} else if (opDesc instanceof AsterixLSMTreeInsertDeleteOperatorDescriptor) {
- operationId = ((AsterixLSMTreeInsertDeleteOperatorDescriptor) opDesc).getIndexName();
- FeedMetaOperatorDescriptor metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc,
- feedPolicy, FeedRuntimeType.STORAGE, operationId);
+ operandId = ((AsterixLSMTreeInsertDeleteOperatorDescriptor) opDesc).getIndexName();
+ metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc, feedPolicyProperties,
+ FeedRuntimeType.STORE, false, operandId);
oldNewOID.put(opDesc.getOperatorId(), metaOp.getOperatorId());
} else if (opDesc instanceof AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor) {
- operationId = ((AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor) opDesc).getIndexName();
- FeedMetaOperatorDescriptor metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc,
- feedPolicy, FeedRuntimeType.STORAGE, operationId);
+ operandId = ((AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor) opDesc).getIndexName();
+ metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc, feedPolicyProperties,
+ FeedRuntimeType.STORE, false, operandId);
oldNewOID.put(opDesc.getOperatorId(), metaOp.getOperatorId());
+
} else {
FeedRuntimeType runtimeType = null;
+ boolean enableSubscriptionMode = false;
+ boolean createMetaOp = true;
+ OperatorDescriptorId opId = 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;
+ IConnectorDescriptor connectorDesc = spec.getOperatorInputMap().get(opDesc.getOperatorId())
+ .get(0);
+ IOperatorDescriptor sourceOp = spec.getProducer(connectorDesc);
+ if (sourceOp instanceof FeedCollectOperatorDescriptor) {
+ runtimeType = preProcessingRequired ? FeedRuntimeType.COMPUTE : FeedRuntimeType.OTHER;
+ enableSubscriptionMode = preProcessingRequired;
+ } else {
+ runtimeType = FeedRuntimeType.OTHER;
+ }
+ } else if (runtimeFactory instanceof EmptyTupleSourceRuntimeFactory) {
+ runtimeType = FeedRuntimeType.ETS;
+ } else {
+ runtimeType = FeedRuntimeType.OTHER;
+ }
+ } else {
+ if (opDesc instanceof AbstractSingleActivityOperatorDescriptor) {
+ runtimeType = FeedRuntimeType.OTHER;
+ } else {
+ opId = altered.createOperatorDescriptorId(opDesc);
+ createMetaOp = false;
}
}
- FeedMetaOperatorDescriptor metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc,
- feedPolicy, runtimeType, operationId);
-
- oldNewOID.put(opDesc.getOperatorId(), metaOp.getOperatorId());
+ if (createMetaOp) {
+ metaOp = new FeedMetaOperatorDescriptor(altered, feedConnectionId, opDesc, feedPolicyProperties,
+ runtimeType, enableSubscriptionMode, operandId);
+ opId = metaOp.getOperatorId();
+ }
+ oldNewOID.put(opDesc.getOperatorId(), opId);
}
}
@@ -240,35 +315,181 @@
}
- public static Triple<IAdapterFactory, ARecordType, AdapterType> getFeedFactoryAndOutput(Feed feed,
- MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
+ public static void increaseCardinality(JobSpecification spec, FeedRuntimeType compute, int requiredCardinality,
+ List<String> newLocations) throws AsterixException {
+ IOperatorDescriptor changingOpDesc = alterJobSpecForComputeCardinality(spec, requiredCardinality);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, changingOpDesc,
+ nChooseK(requiredCardinality, newLocations));
+
+ }
+
+ public static void decreaseComputeCardinality(JobSpecification spec, FeedRuntimeType compute,
+ int requiredCardinality, List<String> currentLocations) throws AsterixException {
+ IOperatorDescriptor changingOpDesc = alterJobSpecForComputeCardinality(spec, requiredCardinality);
+ String[] chosenLocations = nChooseK(requiredCardinality, currentLocations);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, changingOpDesc, chosenLocations);
+ }
+
+ private static IOperatorDescriptor alterJobSpecForComputeCardinality(JobSpecification spec, int requiredCardinality)
+ throws AsterixException {
+ Map<ConnectorDescriptorId, IConnectorDescriptor> connectors = spec.getConnectorMap();
+ Map<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> connectorOpMap = spec
+ .getConnectorOperatorMap();
+
+ IOperatorDescriptor sourceOp = null;
+ IOperatorDescriptor targetOp = null;
+ IConnectorDescriptor connDesc = null;
+ for (Entry<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> entry : connectorOpMap
+ .entrySet()) {
+ ConnectorDescriptorId cid = entry.getKey();
+ sourceOp = entry.getValue().getKey().getKey();
+ if (sourceOp instanceof FeedCollectOperatorDescriptor) {
+ targetOp = entry.getValue().getValue().getKey();
+ if (targetOp instanceof FeedMetaOperatorDescriptor
+ && (((FeedMetaOperatorDescriptor) targetOp).getRuntimeType().equals(FeedRuntimeType.COMPUTE))) {
+ connDesc = connectors.get(cid);
+ break;
+ } else {
+ throw new AsterixException("Incorrect manipulation, feed does not have a compute stage");
+ }
+ }
+ }
+
+ Map<OperatorDescriptorId, List<IConnectorDescriptor>> operatorInputMap = spec.getOperatorInputMap();
+ boolean removed = operatorInputMap.get(targetOp.getOperatorId()).remove(connDesc);
+ if (!removed) {
+ throw new AsterixException("Connector desc not found");
+ }
+ Map<OperatorDescriptorId, List<IConnectorDescriptor>> operatorOutputMap = spec.getOperatorOutputMap();
+ removed = operatorOutputMap.get(sourceOp.getOperatorId()).remove(connDesc);
+ if (!removed) {
+ throw new AsterixException("Connector desc not found");
+ }
+ spec.getConnectorMap().remove(connDesc.getConnectorId());
+ connectorOpMap.remove(connDesc.getConnectorId());
+
+ ITuplePartitionComputerFactory tpcf = new RandomPartitionComputerFactory(requiredCardinality);
+ MToNPartitioningConnectorDescriptor newConnector = new MToNPartitioningConnectorDescriptor(spec, tpcf);
+ spec.getConnectorMap().put(newConnector.getConnectorId(), newConnector);
+ spec.connect(newConnector, sourceOp, 0, targetOp, 0);
+
+ // ==============================================================================
+ Set<Constraint> userConstraints = spec.getUserConstraints();
+ Constraint countConstraint = null;
+ Constraint locationConstraint = null;
+ List<LocationConstraint> locations = new ArrayList<LocationConstraint>();
+ IOperatorDescriptor changingOpDesc = null;
+
+ for (Constraint constraint : userConstraints) {
+ LValueConstraintExpression lexpr = constraint.getLValue();
+ ConstraintExpression cexpr = constraint.getRValue();
+ OperatorDescriptorId opId;
+ switch (lexpr.getTag()) {
+ case PARTITION_COUNT: {
+ opId = ((PartitionCountExpression) lexpr).getOperatorDescriptorId();
+ IOperatorDescriptor opDesc = spec.getOperatorMap().get(opId);
+ if (opDesc instanceof FeedMetaOperatorDescriptor) {
+ FeedRuntimeType runtimeType = ((FeedMetaOperatorDescriptor) opDesc).getRuntimeType();
+ if (runtimeType.equals(FeedRuntimeType.COMPUTE)) {
+ countConstraint = constraint;
+ changingOpDesc = opDesc;
+ }
+ }
+ break;
+ }
+ case PARTITION_LOCATION:
+ opId = ((PartitionLocationExpression) lexpr).getOperatorDescriptorId();
+ IOperatorDescriptor opDesc = spec.getOperatorMap().get(opId);
+ if (opDesc instanceof FeedMetaOperatorDescriptor) {
+ FeedRuntimeType runtimeType = ((FeedMetaOperatorDescriptor) opDesc).getRuntimeType();
+ if (runtimeType.equals(FeedRuntimeType.COMPUTE)) {
+ locationConstraint = constraint;
+ changingOpDesc = opDesc;
+ String location = (String) ((ConstantExpression) cexpr).getValue();
+ LocationConstraint lc = new LocationConstraint();
+ lc.location = location;
+ lc.partition = ((PartitionLocationExpression) lexpr).getPartition();
+ locations.add(lc);
+ }
+ }
+
+ break;
+ }
+ }
+
+ userConstraints.remove(countConstraint);
+ if (locationConstraint != null) {
+ userConstraints.remove(locationConstraint);
+ }
+
+ return changingOpDesc;
+ }
+
+ private static String[] nChooseK(int k, List<String> locations) {
+ String[] result = new String[k];
+ for (int i = 0; i < k; i++) {
+ result[i] = locations.get(i);
+ }
+ return result;
+ }
+
+ private static boolean preProcessingRequired(FeedConnectionId connectionId) {
+ MetadataTransactionContext ctx = null;
+ Feed feed = null;
+ boolean preProcessingRequired = false;
+ try {
+ MetadataManager.INSTANCE.acquireReadLatch();
+ ctx = MetadataManager.INSTANCE.beginTransaction();
+ feed = MetadataManager.INSTANCE.getFeed(ctx, connectionId.getFeedId().getDataverse(), connectionId
+ .getFeedId().getFeedName());
+ preProcessingRequired = feed.getAppliedFunction() != null;
+ MetadataManager.INSTANCE.commitTransaction(ctx);
+ } catch (Exception e) {
+ if (ctx != null) {
+ try {
+ MetadataManager.INSTANCE.abortTransaction(ctx);
+ } catch (Exception abortException) {
+ e.addSuppressed(abortException);
+ throw new IllegalStateException(e);
+ }
+ }
+ } finally {
+ MetadataManager.INSTANCE.releaseReadLatch();
+ }
+ return preProcessingRequired;
+ }
+
+ public static Triple<IFeedAdapterFactory, ARecordType, AdapterType> getPrimaryFeedFactoryAndOutput(
+ PrimaryFeed feed, FeedPolicyAccessor policyAccessor, MetadataTransactionContext mdTxnCtx)
+ throws AlgebricksException {
String adapterName = null;
DatasourceAdapter adapterEntity = null;
String adapterFactoryClassname = null;
- IAdapterFactory adapterFactory = null;
+ IFeedAdapterFactory adapterFactory = null;
ARecordType adapterOutputType = null;
- Triple<IAdapterFactory, ARecordType, AdapterType> feedProps = null;
+ Triple<IFeedAdapterFactory, ARecordType, AdapterType> feedProps = null;
+ AdapterType adapterType = null;
try {
- adapterName = feed.getAdapterName();
+ adapterName = feed.getAdaptorName();
adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
adapterName);
if (adapterEntity == null) {
adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, feed.getDataverseName(), adapterName);
}
-
if (adapterEntity != null) {
+ adapterType = adapterEntity.getType();
adapterFactoryClassname = adapterEntity.getClassname();
- switch (adapterEntity.getType()) {
+ switch (adapterType) {
case INTERNAL:
- adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+ adapterFactory = (IFeedAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
break;
case EXTERNAL:
String[] anameComponents = adapterName.split("#");
String libraryName = anameComponents[0];
ClassLoader cl = ExternalLibraryManager.getLibraryClassLoader(feed.getDataverseName(),
libraryName);
- adapterFactory = (IAdapterFactory) cl.loadClass(adapterFactoryClassname).newInstance();
+ adapterFactory = (IFeedAdapterFactory) cl.loadClass(adapterFactoryClassname).newInstance();
break;
}
} else {
@@ -276,42 +497,90 @@
if (adapterFactoryClassname == null) {
adapterFactoryClassname = adapterName;
}
- adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+ adapterFactory = (IFeedAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+ adapterType = AdapterType.INTERNAL;
}
- Map<String, String> configuration = feed.getAdapterConfiguration();
-
- switch (adapterFactory.getAdapterType()) {
- case TYPED:
- ((ITypedAdapterFactory) adapterFactory).configure(configuration);
- adapterOutputType = ((ITypedAdapterFactory) adapterFactory).getAdapterOutputType();
- break;
- case GENERIC:
- String outputTypeName = configuration.get(IGenericAdapterFactory.KEY_TYPE_NAME);
- if (outputTypeName == null) {
- throw new IllegalArgumentException(
- "You must specify the datatype associated with the incoming data. Datatype is specified by the "
- + IGenericAdapterFactory.KEY_TYPE_NAME + " configuration parameter");
- }
- Datatype datatype = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, feed.getDataverseName(),
- outputTypeName);
- if (datatype == null) {
- throw new Exception("no datatype \"" + outputTypeName + "\" in dataverse \""
- + feed.getDataverseName() + "\"");
- }
- adapterOutputType = (ARecordType) datatype.getDatatype();
- ((IGenericAdapterFactory) adapterFactory).configure(configuration, (ARecordType) adapterOutputType);
- break;
- default:
- throw new IllegalStateException(" Unknown factory type for " + adapterFactoryClassname);
- }
-
- feedProps = new Triple<IAdapterFactory, ARecordType, AdapterType>(adapterFactory, adapterOutputType,
- adapterEntity.getType());
+ Map<String, String> configuration = feed.getAdaptorConfiguration();
+ configuration.putAll(policyAccessor.getFeedPolicy());
+ adapterOutputType = getOutputType(feed, configuration);
+ adapterFactory.configure(configuration, adapterOutputType);
+ feedProps = new Triple<IFeedAdapterFactory, ARecordType, AdapterType>(adapterFactory, adapterOutputType,
+ adapterType);
} catch (Exception e) {
e.printStackTrace();
- throw new AlgebricksException("unable to create adapter " + e);
+ throw new AlgebricksException("unable to create adapter " + e);
}
return feedProps;
}
+
+ private static ARecordType getOutputType(PrimaryFeed feed, Map<String, String> configuration) throws Exception {
+ ARecordType outputType = null;
+ String fqOutputType = configuration.get(IAdapterFactory.KEY_TYPE_NAME);
+
+ if (fqOutputType == null) {
+ throw new IllegalArgumentException("No output type specified");
+ }
+ String[] dataverseAndType = fqOutputType.split("[.]");
+ String dataverseName;
+ String datatypeName;
+
+ if (dataverseAndType.length == 1) {
+ datatypeName = dataverseAndType[0];
+ dataverseName = feed.getDataverseName();
+ } else if (dataverseAndType.length == 2) {
+ dataverseName = dataverseAndType[0];
+ datatypeName = dataverseAndType[1];
+ } else
+ throw new IllegalArgumentException("Invalid value for the parameter " + IAdapterFactory.KEY_TYPE_NAME);
+
+ MetadataTransactionContext ctx = null;
+ MetadataManager.INSTANCE.acquireReadLatch();
+ try {
+ ctx = MetadataManager.INSTANCE.beginTransaction();
+ Datatype t = MetadataManager.INSTANCE.getDatatype(ctx, dataverseName, datatypeName);
+ IAType type = t.getDatatype();
+ if (type.getTypeTag() != ATypeTag.RECORD) {
+ throw new IllegalStateException();
+ }
+ outputType = (ARecordType) t.getDatatype();
+ MetadataManager.INSTANCE.commitTransaction(ctx);
+ } catch (Exception e) {
+ if (ctx != null) {
+ MetadataManager.INSTANCE.abortTransaction(ctx);
+ }
+ throw e;
+ } finally {
+ MetadataManager.INSTANCE.releaseReadLatch();
+ }
+ return outputType;
+ }
+
+ public static String getSecondaryFeedOutput(SecondaryFeed feed, FeedPolicyAccessor policyAccessor,
+ MetadataTransactionContext mdTxnCtx) throws AlgebricksException, MetadataException {
+ String outputType = null;
+ String primaryFeedName = feed.getSourceFeedName();
+ Feed primaryFeed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, feed.getDataverseName(), primaryFeedName);
+ FunctionSignature appliedFunction = primaryFeed.getAppliedFunction();
+ if (appliedFunction == null) {
+ Triple<IFeedAdapterFactory, ARecordType, AdapterType> result = getPrimaryFeedFactoryAndOutput(
+ (PrimaryFeed) primaryFeed, policyAccessor, mdTxnCtx);
+ outputType = result.second.getTypeName();
+ } else {
+ Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, appliedFunction);
+ if (function != null) {
+ if (function.getLanguage().equals(Function.LANGUAGE_AQL)) {
+ throw new NotImplementedException(
+ "Secondary feeds derived from a source feed that has an applied AQL function are not supported yet.");
+ } else {
+ outputType = function.getReturnType();
+ }
+ } else {
+ throw new IllegalArgumentException("Function " + appliedFunction
+ + " associated with source feed not found in Metadata.");
+ }
+ }
+ return outputType;
+ }
+
}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedWorkManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedWorkManager.java
new file mode 100644
index 0000000..731ca56
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/FeedWorkManager.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.metadata.feeds;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import edu.uci.ics.asterix.common.feeds.api.IFeedWork;
+import edu.uci.ics.asterix.common.feeds.api.IFeedWorkEventListener;
+import edu.uci.ics.asterix.common.feeds.api.IFeedWorkManager;
+
+/**
+ * Handles asynchronous execution of feed management related tasks.
+ */
+public class FeedWorkManager implements IFeedWorkManager {
+
+ public static final FeedWorkManager INSTANCE = new FeedWorkManager();
+
+ private final ExecutorService executorService = Executors.newCachedThreadPool();
+
+ private FeedWorkManager() {
+ }
+
+ public void submitWork(IFeedWork work, IFeedWorkEventListener listener) {
+ Runnable runnable = work.getRunnable();
+ try {
+ executorService.execute(runnable);
+ listener.workCompleted(work);
+ } catch (Exception e) {
+ listener.workFailed(work, e);
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedAdapter.java
deleted file mode 100644
index 55abd73..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedAdapter.java
+++ /dev/null
@@ -1,40 +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.feeds;
-
-/**
- * Interface implemented by an adapter that can be controlled or managed by external
- * commands (stop,alter)
- */
-public interface IFeedAdapter extends IDatasourceAdapter {
-
- public enum DataExchangeMode {
- PULL,
- PUSH
- }
-
- /**
- * @return
- */
- public DataExchangeMode getDataExchangeMode();
-
- /**
- * 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/api/IClusterManagementWork.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedAdapterFactory.java
similarity index 64%
copy from asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
copy to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedAdapterFactory.java
index 65ac354..fc6bfe7 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IClusterManagementWork.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IFeedAdapterFactory.java
@@ -12,18 +12,15 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.asterix.metadata.api;
+package edu.uci.ics.asterix.metadata.feeds;
-public interface IClusterManagementWork {
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory;
- public enum WorkType {
- ADD_NODE,
- REMOVE_NODE
- }
+public interface IFeedAdapterFactory extends IAdapterFactory {
- public WorkType getClusterManagementWorkType();
+ public boolean isRecordTrackingEnabled();
- public int getWorkId();
+ public IIntakeProgressTracker createIntakeProgressTracker();
- public IClusterEventsSubscriber getSourceSubscriber();
}
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
deleted file mode 100644
index 47aa8d9..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IGenericAdapterFactory.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.metadata.feeds;
-
-import java.util.List;
-import java.util.Map;
-
-import edu.uci.ics.asterix.metadata.entities.ExternalFile;
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-
-public interface IGenericAdapterFactory extends IAdapterFactory {
-
- public static final String KEY_TYPE_NAME = "type-name";
-
- public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception;
-
- public void setFiles(List<ExternalFile> files) throws AlgebricksException;
-
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IPullBasedFeedAdapter.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IPullBasedFeedAdapter.java
index 50641b0..dbc16d6 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IPullBasedFeedAdapter.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IPullBasedFeedAdapter.java
@@ -14,6 +14,8 @@
*/
package edu.uci.ics.asterix.metadata.feeds;
+import edu.uci.ics.asterix.common.feeds.api.IFeedAdapter;
+
public interface IPullBasedFeedAdapter extends IFeedAdapter {
/**
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ITypedAdapterFactory.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ITypedAdapterFactory.java
index 6faa44b..de2086e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ITypedAdapterFactory.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/ITypedAdapterFactory.java
@@ -16,6 +16,7 @@
import java.util.Map;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory;
import edu.uci.ics.asterix.om.types.ARecordType;
public interface ITypedAdapterFactory extends IAdapterFactory {
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
deleted file mode 100644
index 6cfdc7f..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/IngestionRuntime.java
+++ /dev/null
@@ -1,34 +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.feeds;
-
-import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
-import edu.uci.ics.asterix.common.feeds.FeedRuntime;
-
-public class IngestionRuntime extends FeedRuntime {
-
- private AdapterRuntimeManager adapterRuntimeManager;
-
- public IngestionRuntime(FeedConnectionId feedId, int partition, FeedRuntimeType feedRuntimeType,
- AdapterRuntimeManager adapterRuntimeManager) {
- super(feedId, partition, feedRuntimeType);
- this.adapterRuntimeManager = adapterRuntimeManager;
- }
-
- public AdapterRuntimeManager getAdapterRuntimeManager() {
- return adapterRuntimeManager;
- }
-
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/PrepareStallMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/PrepareStallMessage.java
new file mode 100644
index 0000000..8c6d54d
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/PrepareStallMessage.java
@@ -0,0 +1,66 @@
+/*
+ * 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 org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.message.FeedMessage;
+
+/**
+ * A feed control message indicating the need to end the feed. This message is dispatched
+ * to all locations that host an operator involved in the feed pipeline.
+ */
+public class PrepareStallMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedConnectionId connectionId;
+
+ private final int computePartitionsRetainLimit;
+
+ public PrepareStallMessage(FeedConnectionId connectionId, int computePartitionsRetainLimit) {
+ super(MessageType.PREPARE_STALL);
+ this.connectionId = connectionId;
+ this.computePartitionsRetainLimit = computePartitionsRetainLimit;
+ }
+
+ @Override
+ public String toString() {
+ return MessageType.PREPARE_STALL.name() + " " + connectionId;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ obj.put(FeedConstants.MessageConstants.COMPUTE_PARTITION_RETAIN_LIMIT, computePartitionsRetainLimit);
+ return obj;
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public int getComputePartitionsRetainLimit() {
+ return computePartitionsRetainLimit;
+ }
+
+}
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
index d7e2e2f..404e14b 100644
--- 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
@@ -31,8 +31,7 @@
private final String host;
private final int port;
private final LinkedBlockingQueue<String> outbox;
-
- private ExecutorService executorService = Executors.newFixedThreadPool(10);
+ private final ExecutorService executorService = Executors.newFixedThreadPool(10);
private RemoteMessageListenerServer listenerServer;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/SocketMessageListener.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/SocketMessageListener.java
new file mode 100644
index 0000000..515b78e
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/SocketMessageListener.java
@@ -0,0 +1,158 @@
+/*
+ * 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.InputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.nio.CharBuffer;
+import java.util.concurrent.Executor;
+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.common.feeds.api.IMessageReceiver;
+
+/**
+ * Listens for messages at a configured port and redirects them to a
+ * an instance of {@code IMessageReceiver}.
+ * Messages may arrive in parallel from multiple senders. Each sender is handled by
+ * a respective instance of {@code ClientHandler}.
+ */
+public class SocketMessageListener {
+
+ private static final Logger LOGGER = Logger.getLogger(SocketMessageListener.class.getName());
+
+ private final int port;
+ private final IMessageReceiver<String> messageReceiver;
+ private final MessageListenerServer listenerServer;
+
+ private ExecutorService executorService = Executors.newFixedThreadPool(10);
+
+ public SocketMessageListener(int port, IMessageReceiver<String> messageReceiver) {
+ this.port = port;
+ this.messageReceiver = messageReceiver;
+ this.listenerServer = new MessageListenerServer(port, messageReceiver);
+ }
+
+ public void stop() throws IOException {
+ listenerServer.stop();
+ messageReceiver.close(false);
+ if (!executorService.isShutdown()) {
+ executorService.shutdownNow();
+ }
+ }
+
+ public void start() {
+ messageReceiver.start();
+ executorService.execute(listenerServer);
+ }
+
+ private static class MessageListenerServer implements Runnable {
+
+ private final int port;
+ private final IMessageReceiver<String> messageReceiver;
+ private ServerSocket server;
+ private final Executor executor;
+
+ public MessageListenerServer(int port, IMessageReceiver<String> messageReceiver) {
+ this.port = port;
+ this.messageReceiver = messageReceiver;
+ this.executor = Executors.newCachedThreadPool();
+ }
+
+ public void stop() throws IOException {
+ server.close();
+ }
+
+ @Override
+ public void run() {
+ Socket client = null;
+ try {
+ server = new ServerSocket(port);
+ while (true) {
+ client = server.accept();
+ ClientHandler handler = new ClientHandler(client, messageReceiver);
+ executor.execute(handler);
+ }
+ } 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 ClientHandler implements Runnable {
+
+ private static final char EOL = (char) "\n".getBytes()[0];
+
+ private final Socket client;
+ private final IMessageReceiver<String> messageReceiver;
+
+ public ClientHandler(Socket client, IMessageReceiver<String> messageReceiver) {
+ this.client = client;
+ this.messageReceiver = messageReceiver;
+ }
+
+ @Override
+ public void run() {
+ try {
+ 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(), 0, buffer.limit());
+ messageReceiver.sendMessage(s + "\n");
+ buffer.position(0);
+ buffer.limit(5000);
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Unable to process mesages from client" + client);
+ }
+ } finally {
+ if (client != null) {
+ try {
+ client.close();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+ }
+
+ }
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/TerminateDataFlowMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/TerminateDataFlowMessage.java
new file mode 100644
index 0000000..c22d2e1
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/TerminateDataFlowMessage.java
@@ -0,0 +1,50 @@
+/*
+ * 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 org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.message.FeedMessage;
+
+public class TerminateDataFlowMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final FeedConnectionId connectionId;
+
+ public TerminateDataFlowMessage(FeedConnectionId connectionId) {
+ super(MessageType.TERMINATE_FLOW);
+ this.connectionId = connectionId;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ return obj;
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/XAQLFeedMessage.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/XAQLFeedMessage.java
new file mode 100644
index 0000000..e36ea2b
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/feeds/XAQLFeedMessage.java
@@ -0,0 +1,64 @@
+/*
+ * 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 org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.feeds.FeedConnectionId;
+import edu.uci.ics.asterix.common.feeds.FeedConstants;
+import edu.uci.ics.asterix.common.feeds.message.FeedMessage;
+
+/**
+ * A feed control message indicating the need to execute a give AQL.
+ */
+public class XAQLFeedMessage extends FeedMessage {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String aql;
+ private final FeedConnectionId connectionId;
+
+ public XAQLFeedMessage(FeedConnectionId connectionId, String aql) {
+ super(MessageType.XAQL);
+ this.connectionId = connectionId;
+ this.aql = aql;
+ }
+
+ @Override
+ public String toString() {
+ return messageType.name() + " " + connectionId + " [" + aql + "] ";
+ }
+
+ public FeedConnectionId getConnectionId() {
+ return connectionId;
+ }
+
+ public String getAql() {
+ return aql;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject obj = new JSONObject();
+ obj.put(FeedConstants.MessageConstants.MESSAGE_TYPE, messageType.name());
+ obj.put(FeedConstants.MessageConstants.DATAVERSE, connectionId.getFeedId().getDataverse());
+ obj.put(FeedConstants.MessageConstants.FEED, connectionId.getFeedId().getFeedName());
+ obj.put(FeedConstants.MessageConstants.DATASET, connectionId.getDatasetName());
+ obj.put(FeedConstants.MessageConstants.AQL, aql);
+ return obj;
+ }
+
+}
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 086caf1..8165c9c 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
@@ -38,8 +38,8 @@
addMetadataBuiltinFunctions();
AsterixBuiltinFunctions.addUnnestFun(AsterixBuiltinFunctions.DATASET, false);
AsterixBuiltinFunctions.addDatasetFunction(AsterixBuiltinFunctions.DATASET);
- AsterixBuiltinFunctions.addUnnestFun(AsterixBuiltinFunctions.FEED_INGEST, false);
- AsterixBuiltinFunctions.addDatasetFunction(AsterixBuiltinFunctions.FEED_INGEST);
+ AsterixBuiltinFunctions.addUnnestFun(AsterixBuiltinFunctions.FEED_COLLECT, false);
+ AsterixBuiltinFunctions.addDatasetFunction(AsterixBuiltinFunctions.FEED_COLLECT);
AsterixBuiltinFunctions.addUnnestFun(AsterixBuiltinFunctions.FEED_INTERCEPT, false);
AsterixBuiltinFunctions.addDatasetFunction(AsterixBuiltinFunctions.FEED_INTERCEPT);
}
@@ -89,17 +89,17 @@
}
}, true);
- AsterixBuiltinFunctions.addPrivateFunction(AsterixBuiltinFunctions.FEED_INGEST, new IResultTypeComputer() {
+ AsterixBuiltinFunctions.addPrivateFunction(AsterixBuiltinFunctions.FEED_COLLECT, new IResultTypeComputer() {
@Override
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());
+ if (f.getArguments().size() != AsterixBuiltinFunctions.FEED_COLLECT.getArity()) {
+ throw new AlgebricksException("Incorrect number of arguments -> arity is "
+ + AsterixBuiltinFunctions.FEED_COLLECT.getArity() + ", not " + f.getArguments().size());
}
- ILogicalExpression a1 = f.getArguments().get(1).getValue();
+ ILogicalExpression a1 = f.getArguments().get(5).getValue();
IAType t1 = (IAType) env.getType(a1);
if (t1.getTypeTag() == ATypeTag.ANY) {
return BuiltinType.ANY;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/MetadataLockManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/MetadataLockManager.java
index 436fe4d..e456f89 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/MetadataLockManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/MetadataLockManager.java
@@ -15,6 +15,7 @@
private final ConcurrentHashMap<String, ReentrantReadWriteLock> functionsLocks;
private final ConcurrentHashMap<String, ReentrantReadWriteLock> nodeGroupsLocks;
private final ConcurrentHashMap<String, ReentrantReadWriteLock> feedsLocks;
+ private final ConcurrentHashMap<String, ReentrantReadWriteLock> feedPolicyLocks;
private final ConcurrentHashMap<String, ReentrantReadWriteLock> compactionPolicyLocks;
private final ConcurrentHashMap<String, ReentrantReadWriteLock> dataTypeLocks;
@@ -24,6 +25,7 @@
functionsLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
nodeGroupsLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
feedsLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
+ feedPolicyLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
compactionPolicyLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
dataTypeLocks = new ConcurrentHashMap<String, ReentrantReadWriteLock>();
}
@@ -205,6 +207,19 @@
public void releaseFeedWriteLock(String feedName) {
feedsLocks.get(feedName).writeLock().unlock();
}
+
+ public void acquireFeedPolicyWriteLock(String policyName) {
+ ReentrantReadWriteLock fLock = feedPolicyLocks.get(policyName);
+ if (fLock == null) {
+ feedPolicyLocks.putIfAbsent(policyName, new ReentrantReadWriteLock());
+ fLock = feedPolicyLocks.get(policyName);
+ }
+ fLock.writeLock().lock();
+ }
+
+ public void releaseFeedPolicyWriteLock(String policyName) {
+ feedPolicyLocks.get(policyName).writeLock().unlock();
+ }
public void acquireCompactionPolicyReadLock(String compactionPolicyName) {
ReentrantReadWriteLock compactionPolicyLock = compactionPolicyLocks.get(compactionPolicyName);
@@ -411,6 +426,16 @@
releaseFeedWriteLock(feedFullyQualifiedName);
releaseDataverseReadLock(dataverseName);
}
+
+ public void dropFeedPolicyBegin(String dataverseName, String policyName) {
+ releaseFeedWriteLock(policyName);
+ releaseDataverseReadLock(dataverseName);
+ }
+
+ public void dropFeedPolicyEnd(String dataverseName, String policyName) {
+ releaseFeedWriteLock(policyName);
+ releaseDataverseReadLock(dataverseName);
+ }
public void createFeedBegin(String dataverseName, String feedFullyQualifiedName) {
acquireDataverseReadLock(dataverseName);
@@ -434,6 +459,16 @@
releaseDataverseReadLock(dataverseName);
}
+ public void createFeedPolicyBegin(String dataverseName, String policyName) {
+ acquireDataverseReadLock(dataverseName);
+ acquireFeedPolicyWriteLock(policyName);
+ }
+
+ public void createFeedPolicyEnd(String dataverseName, String policyName) {
+ releaseFeedPolicyWriteLock(policyName);
+ releaseDataverseReadLock(dataverseName);
+ }
+
public void disconnectFeedBegin(String dataverseName, String datasetFullyQualifiedName,
String feedFullyQualifiedName) {
acquireDataverseReadLock(dataverseName);
@@ -446,6 +481,19 @@
releaseDatasetReadLock(datasetFullyQualifiedName);
releaseDataverseReadLock(dataverseName);
}
+
+ public void subscribeFeedBegin(String dataverseName, String datasetFullyQualifiedName,
+ String feedFullyQualifiedName) {
+ acquireDataverseReadLock(dataverseName);
+ acquireDatasetReadLock(datasetFullyQualifiedName);
+ acquireFeedReadLock(feedFullyQualifiedName);
+ }
+
+ public void subscribeFeedEnd(String dataverseName, String datasetFullyQualifiedName, String feedFullyQualifiedName) {
+ releaseFeedReadLock(feedFullyQualifiedName);
+ releaseDatasetReadLock(datasetFullyQualifiedName);
+ releaseDataverseReadLock(dataverseName);
+ }
public void compactBegin(String dataverseName, String datasetFullyQualifiedName) {
acquireDataverseReadLock(dataverseName);
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 cfecd9c..fa99064 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
@@ -273,8 +273,8 @@
"string-join", 2);
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", 3);
+ public final static FunctionIdentifier FEED_COLLECT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+ "feed-collect", 6);
public final static FunctionIdentifier FEED_INTERCEPT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
"feed-intercept", 1);
@@ -1253,14 +1253,14 @@
static {
datasetFunctions.add(getAsterixFunctionInfo(DATASET));
- datasetFunctions.add(getAsterixFunctionInfo(FEED_INGEST));
+ datasetFunctions.add(getAsterixFunctionInfo(FEED_COLLECT));
datasetFunctions.add(getAsterixFunctionInfo(FEED_INTERCEPT));
datasetFunctions.add(getAsterixFunctionInfo(INDEX_SEARCH));
}
static {
addUnnestFun(DATASET, false);
- addUnnestFun(FEED_INGEST, false);
+ addUnnestFun(FEED_COLLECT, false);
addUnnestFun(FEED_INTERCEPT, false);
addUnnestFun(RANGE, true);
addUnnestFun(SCAN_COLLECTION, false);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/ARecordPointable.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/ARecordPointable.java
index eb4a027..7d98846 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/ARecordPointable.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/ARecordPointable.java
@@ -275,6 +275,10 @@
public List<IVisitablePointable> getFieldValues() {
return fieldValues;
}
+
+ public ARecordType getInputRecordType(){
+ return inputRecType;
+ }
@Override
public <R, T> R accept(IVisitablePointableVisitor<R, T> vistor, T tag) throws AsterixException {
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 cef7937..a24f2de 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
@@ -18,6 +18,7 @@
import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
+import edu.uci.ics.asterix.common.config.AsterixFeedProperties;
import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
@@ -46,6 +47,7 @@
private AsterixMetadataProperties metadataProperties;
private AsterixStorageProperties storageProperties;
private AsterixTransactionProperties txnProperties;
+ private AsterixFeedProperties feedProperties;
private IHyracksClientConnection hcc;
@@ -59,6 +61,7 @@
INSTANCE.metadataProperties = new AsterixMetadataProperties(propertiesAccessor);
INSTANCE.storageProperties = new AsterixStorageProperties(propertiesAccessor);
INSTANCE.txnProperties = new AsterixTransactionProperties(propertiesAccessor);
+ INSTANCE.feedProperties = new AsterixFeedProperties(propertiesAccessor);
INSTANCE.hcc = hcc;
Logger.getLogger("edu.uci.ics").setLevel(INSTANCE.externalProperties.getLogLevel());
}
@@ -102,6 +105,11 @@
return externalProperties;
}
+ @Override
+ public AsterixFeedProperties getFeedProperties() {
+ return feedProperties;
+ }
+
public IHyracksClientConnection getHcc() {
return hcc;
}
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 8e393d7..de3b1b9 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
@@ -28,6 +28,7 @@
import javax.xml.bind.JAXBException;
import javax.xml.bind.Unmarshaller;
+import edu.uci.ics.asterix.common.api.IClusterManagementWork.ClusterState;
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;
@@ -70,12 +71,9 @@
}
}
- public enum State {
- ACTIVE,
- UNUSABLE
- }
+
- private State state = State.UNUSABLE;
+ private ClusterState state = ClusterState.UNUSABLE;
public synchronized void removeNCConfiguration(String nodeId) {
// state = State.UNUSABLE;
@@ -87,7 +85,7 @@
ncConfiguration.put(nodeId, configuration);
if (ncConfiguration.keySet().size() == AsterixAppContextInfo.getInstance().getMetadataProperties()
.getNodeNames().size()) {
- state = State.ACTIVE;
+ state = ClusterState.ACTIVE;
}
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(" Registering configuration parameters for node id " + nodeId);
@@ -129,7 +127,7 @@
return ncConfig.get(IO_DEVICES).split(",");
}
- public State getState() {
+ public ClusterState getState() {
return state;
}
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 3871aef..8fdd14c 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
@@ -20,6 +20,7 @@
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
@@ -309,8 +310,8 @@
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.TimeFromDatetimeDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.TimeFromUnixTimeInMsDescriptor;
import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.YearMonthDurationComparatorDecriptor;
-import edu.uci.ics.asterix.runtime.operators.file.AdmSchemafullRecordParserFactory;
-import edu.uci.ics.asterix.runtime.operators.file.NtDelimitedDataTupleParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
import edu.uci.ics.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
import edu.uci.ics.asterix.runtime.unnestingfunctions.std.RangeDescriptor;
import edu.uci.ics.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor;
@@ -1081,23 +1082,26 @@
@Override
public ITupleParserFactory createTupleParser(ARecordType recType, boolean delimitedFormat, char delimiter,
char quote, boolean hasHeader) {
+ Map<String, String> conf = new HashMap<String, String>();
+ AsterixTupleParserFactory.InputDataFormat inputFormat = null;
if (delimitedFormat) {
- int n = recType.getFieldTypes().length;
- IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
- for (int i = 0; i < n; i++) {
- ATypeTag tag = recType.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 new NtDelimitedDataTupleParserFactory(recType, fieldParserFactories, delimiter, quote, hasHeader);
+ conf.put(AsterixTupleParserFactory.KEY_FORMAT, AsterixTupleParserFactory.FORMAT_DELIMITED_TEXT);
+ conf.put(AsterixTupleParserFactory.KEY_DELIMITER, "" + delimiter);
+ inputFormat = InputDataFormat.DELIMITED;
} else {
- return new AdmSchemafullRecordParserFactory(recType);
+ conf.put(AsterixTupleParserFactory.KEY_FORMAT, AsterixTupleParserFactory.FORMAT_ADM);
+ inputFormat = InputDataFormat.ADM;
}
+
+ if (hasHeader) {
+ conf.put(AsterixTupleParserFactory.HAS_HEADER,
+ hasHeader ? Boolean.TRUE.toString() : Boolean.FALSE.toString());
+ }
+ conf.put(AsterixTupleParserFactory.KEY_QUOTE, "" + quote);
+ return new AsterixTupleParserFactory(conf, recType, inputFormat);
}
+
@Override
public INullWriterFactory getNullWriterFactory() {
return AqlNullWriterFactory.INSTANCE;
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 d9103ef..d42e0d9 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
@@ -20,18 +20,16 @@
import java.util.logging.Logger;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.parse.ITupleForwardPolicy;
import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
/**
- * An Abstract class implementation for ITupleParser. It provides common
+ * An abstract class implementation for ITupleParser. It provides common
* functionality involved in parsing data in an external format and packing
* frames with formed tuples.
*/
@@ -41,37 +39,34 @@
protected ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
protected DataOutput dos = tb.getDataOutput();
- protected final FrameTupleAppender appender;
protected final ARecordType recType;
protected final IHyracksTaskContext ctx;
- protected String filename;
public AbstractTupleParser(IHyracksTaskContext ctx, ARecordType recType) throws HyracksDataException {
- appender = new FrameTupleAppender(new VSizeFrame(ctx));
this.recType = recType;
this.ctx = ctx;
}
- public void setFilename(String filename) {
- this.filename = filename;
- }
-
public abstract IDataParser getDataParser();
+ public abstract ITupleForwardPolicy getTupleParserPolicy();
+
@Override
public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
IDataParser parser = getDataParser();
+ ITupleForwardPolicy policy = getTupleParserPolicy();
try {
parser.initialize(in, recType, true);
+ policy.initialize(ctx, writer);
while (true) {
tb.reset();
if (!parser.parse(tb.getDataOutput())) {
break;
}
tb.addFieldEndOffset();
- addTupleToFrame(writer);
+ policy.addTuple(tb);
}
- appender.flush(writer, true);
+ policy.close();
} catch (AsterixException ae) {
throw new HyracksDataException(ae);
} catch (IOException ioe) {
@@ -79,15 +74,4 @@
}
}
- protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
- if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- appender.flush(writer, true);
- if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- throw new IllegalStateException("Tuple size(" + tb.getSize() + ") is greater than frame size("
- + AsterixAppContextInfo.getInstance().getCompilerProperties().getFrameSize() + ")");
- }
- }
-
- }
-
}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmSchemafullRecordParserFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmSchemafullRecordParserFactory.java
deleted file mode 100644
index 2ce0e61..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmSchemafullRecordParserFactory.java
+++ /dev/null
@@ -1,41 +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.runtime.operators.file;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
-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;
-
-/**
- * A Tuple parser factory for creating a tuple parser capable of parsing
- * ADM data.
- */
-public class AdmSchemafullRecordParserFactory implements ITupleParserFactory {
-
- private static final long serialVersionUID = 1L;
- protected ARecordType recType;
-
- public AdmSchemafullRecordParserFactory(ARecordType recType) {
- this.recType = recType;
- }
-
- @Override
- public ITupleParser createTupleParser(final IHyracksTaskContext ctx) throws HyracksDataException {
- return new AdmTupleParser(ctx, recType);
- }
-
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
deleted file mode 100644
index 8aab2db..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
+++ /dev/null
@@ -1,36 +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.runtime.operators.file;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * An extension of AbstractTupleParser that provides functionality for
- * parsing delimited files.
- */
-public class AdmTupleParser extends AbstractTupleParser {
-
- public AdmTupleParser(IHyracksTaskContext ctx, ARecordType recType) throws HyracksDataException {
- super(ctx, recType);
- }
-
- @Override
- public IDataParser getDataParser() {
- return new ADMDataParser(filename);
- }
-
-}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AsterixTupleParserFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AsterixTupleParserFactory.java
new file mode 100644
index 0000000..f1bd0f6
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AsterixTupleParserFactory.java
@@ -0,0 +1,254 @@
+package edu.uci.ics.asterix.runtime.operators.file;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.FeedPolicyAccessor;
+import edu.uci.ics.asterix.common.parse.ITupleForwardPolicy;
+import edu.uci.ics.asterix.common.parse.ITupleForwardPolicy.TupleForwardPolicyType;
+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.IAType;
+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.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 class AsterixTupleParserFactory implements ITupleParserFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static enum InputDataFormat {
+ ADM,
+ DELIMITED,
+ UNKNOWN
+ }
+
+ public static final String HAS_HEADER = "has.header";
+ public static final String KEY_FORMAT = "format";
+ public static final String FORMAT_ADM = "adm";
+ public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
+ public static final String FORMAT_BINARY = "binary";
+
+ public static final String KEY_PATH = "path";
+ public static final String KEY_SOURCE_DATATYPE = "type-name";
+ public static final String KEY_DELIMITER = "delimiter";
+ public static final String KEY_PARSER_FACTORY = "parser";
+ public static final String KEY_HEADER = "header";
+ public static final String KEY_QUOTE = "quote";
+ public static final String TIME_TRACKING = "time.tracking";
+ public static final String DEFAULT_QUOTE = "\"";
+ public static final String AT_LEAST_ONE_SEMANTICS = FeedPolicyAccessor.AT_LEAST_ONE_SEMANTICS;
+ public static final String NODE_RESOLVER_FACTORY_PROPERTY = "node.Resolver";
+ public static final String DEFAULT_DELIMITER = ",";
+
+ private static Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = initializeValueParserFactoryMap();
+
+ private static Map<ATypeTag, IValueParserFactory> initializeValueParserFactoryMap() {
+ Map<ATypeTag, IValueParserFactory> m = new HashMap<ATypeTag, IValueParserFactory>();
+ m.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
+ m.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
+ m.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
+ m.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
+ m.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
+ return m;
+ }
+
+ private final ARecordType recordType;
+ private final Map<String, String> configuration;
+ private final InputDataFormat inputDataFormat;
+
+ public AsterixTupleParserFactory(Map<String, String> configuration, ARecordType recType, InputDataFormat dataFormat) {
+ this.recordType = recType;
+ this.configuration = configuration;
+ this.inputDataFormat = dataFormat;
+ }
+
+ @Override
+ public ITupleParser createTupleParser(IHyracksTaskContext ctx) throws HyracksDataException {
+ ITupleParser tupleParser = null;
+ try {
+ String parserFactoryClassname = (String) configuration.get(KEY_PARSER_FACTORY);
+ ITupleParserFactory parserFactory = null;
+ if (parserFactoryClassname != null) {
+ parserFactory = (ITupleParserFactory) Class.forName(parserFactoryClassname).newInstance();
+ tupleParser = parserFactory.createTupleParser(ctx);
+ } else {
+ IDataParser dataParser = null;
+ dataParser = createDataParser(ctx);
+ ITupleForwardPolicy policy = getTupleParserPolicy(configuration);
+ policy.configure(configuration);
+ tupleParser = new GenericTupleParser(ctx, recordType, dataParser, policy);
+ }
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ return tupleParser;
+ }
+
+ private static class GenericTupleParser extends AbstractTupleParser {
+
+ private final IDataParser dataParser;
+
+ private final ITupleForwardPolicy policy;
+
+ public GenericTupleParser(IHyracksTaskContext ctx, ARecordType recType, IDataParser dataParser,
+ ITupleForwardPolicy policy) throws HyracksDataException {
+ super(ctx, recType);
+ this.dataParser = dataParser;
+ this.policy = policy;
+ }
+
+ @Override
+ public IDataParser getDataParser() {
+ return dataParser;
+ }
+
+ @Override
+ public ITupleForwardPolicy getTupleParserPolicy() {
+ return policy;
+ }
+
+ }
+
+ private IDataParser createDataParser(IHyracksTaskContext ctx) throws Exception {
+ IDataParser dataParser = null;
+ switch (inputDataFormat) {
+ case ADM:
+ dataParser = new ADMDataParser();
+ break;
+ case DELIMITED:
+ dataParser = configureDelimitedDataParser(ctx);
+ break;
+ case UNKNOWN:
+ String specifiedFormat = (String) configuration.get(KEY_FORMAT);
+ if (specifiedFormat == null) {
+ throw new IllegalArgumentException(" Unspecified data format");
+ } else {
+ if (FORMAT_ADM.equalsIgnoreCase(specifiedFormat.toUpperCase())) {
+ dataParser = new ADMDataParser();
+ } else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat.toUpperCase())) {
+ dataParser = configureDelimitedDataParser(ctx);
+ } else {
+ throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT)
+ + " not supported");
+ }
+ }
+ }
+ return dataParser;
+ }
+
+ public static ITupleForwardPolicy getTupleParserPolicy(Map<String, String> configuration) {
+ ITupleForwardPolicy policy = null;
+ ITupleForwardPolicy.TupleForwardPolicyType policyType = null;
+ String propValue = configuration.get(ITupleForwardPolicy.PARSER_POLICY);
+ if (propValue == null) {
+ policyType = TupleForwardPolicyType.FRAME_FULL;
+ } else {
+ policyType = TupleForwardPolicyType.valueOf(propValue.trim().toUpperCase());
+ }
+ switch (policyType) {
+ case FRAME_FULL:
+ policy = new FrameFullTupleForwardPolicy();
+ break;
+ case COUNTER_TIMER_EXPIRED:
+ policy = new CounterTimerTupleForwardPolicy();
+ break;
+ case RATE_CONTROLLED:
+ policy = new RateControlledTupleForwardPolicy();
+ break;
+ }
+ return policy;
+ }
+
+ private IDataParser configureDelimitedDataParser(IHyracksTaskContext ctx) throws AsterixException {
+ IValueParserFactory[] valueParserFactories = getValueParserFactories();
+ Character delimiter = getDelimiter(configuration);
+ char quote = getQuote(configuration, delimiter);
+ boolean hasHeader = hasHeader();
+ return new DelimitedDataParser(recordType, valueParserFactories, delimiter, quote, hasHeader);
+ }
+
+
+ private boolean hasHeader() {
+ String value = configuration.get(KEY_HEADER);
+ if (value != null) {
+ return Boolean.valueOf(value);
+ }
+ return false;
+ }
+
+ private IValueParserFactory[] getValueParserFactories() {
+ 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 = valueParserFactoryMap.get(tag);
+ if (vpf == null) {
+ throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
+ }
+ fieldParserFactories[i] = vpf;
+ }
+ return fieldParserFactories;
+ }
+
+ // Get a delimiter from the given configuration
+ public static char getDelimiter(Map<String, String> configuration) throws AsterixException {
+ String delimiterValue = configuration.get(AsterixTupleParserFactory.KEY_DELIMITER);
+ if (delimiterValue == null) {
+ delimiterValue = AsterixTupleParserFactory.DEFAULT_DELIMITER;
+ } else if (delimiterValue.length() != 1) {
+ throw new AsterixException("'" + delimiterValue
+ + "' is not a valid delimiter. The length of a delimiter should be 1.");
+ }
+ return delimiterValue.charAt(0);
+ }
+
+ // Get a quote from the given configuration when the delimiter is given
+ // Need to pass delimiter to check whether they share the same character
+ public static char getQuote(Map<String, String> configuration, char delimiter) throws AsterixException {
+ String quoteValue = configuration.get(AsterixTupleParserFactory.KEY_QUOTE);
+ if (quoteValue == null) {
+ quoteValue = AsterixTupleParserFactory.DEFAULT_QUOTE;
+ } else if (quoteValue.length() != 1) {
+ throw new AsterixException("'" + quoteValue + "' is not a valid quote. The length of a quote should be 1.");
+ }
+
+ // Since delimiter (char type value) can't be null,
+ // we only check whether delimiter and quote use the same character
+ if (quoteValue.charAt(0) == delimiter) {
+ throw new AsterixException("Quote '" + quoteValue + "' cannot be used with the delimiter '" + delimiter
+ + "'. ");
+ }
+
+ return quoteValue.charAt(0);
+ }
+
+ // Get the header flag
+ public static boolean getHasHeader(Map<String, String> configuration) {
+ return Boolean.parseBoolean(configuration.get(AsterixTupleParserFactory.KEY_HEADER));
+ }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/CounterTimerTupleForwardPolicy.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/CounterTimerTupleForwardPolicy.java
new file mode 100644
index 0000000..261eb13
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/CounterTimerTupleForwardPolicy.java
@@ -0,0 +1,140 @@
+package edu.uci.ics.asterix.runtime.operators.file;
+
+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.parse.ITupleForwardPolicy;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+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;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+
+public class CounterTimerTupleForwardPolicy implements ITupleForwardPolicy {
+
+ public static final String BATCH_SIZE = "batch-size";
+ public static final String BATCH_INTERVAL = "batch-interval";
+
+ private static final Logger LOGGER = Logger.getLogger(CounterTimerTupleForwardPolicy.class.getName());
+
+ private FrameTupleAppender appender;
+ private IFrame frame;
+ private IFrameWriter writer;
+ private int batchSize;
+ private long batchInterval;
+ private int tuplesInFrame = 0;
+ private TimeBasedFlushTask flushTask;
+ private Timer timer;
+ private Object lock = new Object();
+ private boolean activeTimer = false;
+
+ public void configure(Map<String, String> configuration) {
+ String propValue = (String) configuration.get(BATCH_SIZE);
+ if (propValue != null) {
+ batchSize = Integer.parseInt(propValue);
+ } else {
+ batchSize = -1;
+ }
+
+ propValue = (String) configuration.get(BATCH_INTERVAL);
+ if (propValue != null) {
+ batchInterval = Long.parseLong(propValue);
+ activeTimer = true;
+ }
+ }
+
+ public void initialize(IHyracksTaskContext ctx, IFrameWriter writer) throws HyracksDataException {
+ this.appender = new FrameTupleAppender();
+ this.frame = new VSizeFrame(ctx);
+ appender.reset(frame, true);
+ this.writer = writer;
+ if (activeTimer) {
+ this.timer = new Timer();
+ this.flushTask = new TimeBasedFlushTask(writer, lock);
+ timer.scheduleAtFixedRate(flushTask, 0, batchInterval);
+ }
+ }
+
+ public void addTuple(ArrayTupleBuilder tb) throws HyracksDataException {
+ if (activeTimer) {
+ synchronized (lock) {
+ addTupleToFrame(tb);
+ }
+ } else {
+ addTupleToFrame(tb);
+ }
+ tuplesInFrame++;
+ }
+
+ private void addTupleToFrame(ArrayTupleBuilder tb) throws HyracksDataException {
+ if (tuplesInFrame == batchSize || !appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("flushing frame containg (" + tuplesInFrame + ") tuples");
+ }
+ FrameUtils.flushFrame(frame.getBuffer(), writer);
+ tuplesInFrame = 0;
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+ public void close() throws HyracksDataException {
+ if (appender.getTupleCount() > 0) {
+ if (activeTimer) {
+ synchronized (lock) {
+ FrameUtils.flushFrame(frame.getBuffer(), writer);
+ }
+ } else {
+ FrameUtils.flushFrame(frame.getBuffer(), writer);
+ }
+ }
+
+ if (timer != null) {
+ timer.cancel();
+ }
+ }
+
+ 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) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("TTL expired flushing frame (" + tuplesInFrame + ")");
+ }
+ synchronized (lock) {
+ FrameUtils.flushFrame(frame.getBuffer(), writer);
+ appender.reset(frame, true);
+ tuplesInFrame = 0;
+ }
+ }
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+ }
+
+ }
+
+ @Override
+ public TupleForwardPolicyType getType() {
+ return TupleForwardPolicyType.COUNTER_TIMER_EXPIRED;
+ }
+
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataTupleParser.java
deleted file mode 100644
index 42e336d..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataTupleParser.java
+++ /dev/null
@@ -1,41 +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.runtime.operators.file;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-
-/**
- * An extension of AbstractTupleParser that provides functionality for
- * parsing delimited files.
- */
-public class DelimitedDataTupleParser extends AbstractTupleParser {
-
- private final DelimitedDataParser dataParser;
-
- public DelimitedDataTupleParser(IHyracksTaskContext ctx, ARecordType recType,
- IValueParserFactory[] valueParserFactories, char fieldDelimter, char quote, boolean hasHeader) throws HyracksDataException {
- super(ctx, recType);
- dataParser = new DelimitedDataParser(recType, valueParserFactories, fieldDelimter, quote, hasHeader);
- }
-
- @Override
- public IDataParser getDataParser() {
- return dataParser;
- }
-
-}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/FrameFullTupleForwardPolicy.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/FrameFullTupleForwardPolicy.java
new file mode 100644
index 0000000..6ce3dfa
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/FrameFullTupleForwardPolicy.java
@@ -0,0 +1,58 @@
+package edu.uci.ics.asterix.runtime.operators.file;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.parse.ITupleForwardPolicy;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+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;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+
+public class FrameFullTupleForwardPolicy implements ITupleForwardPolicy {
+
+ private FrameTupleAppender appender;
+ private IFrame frame;
+ private IFrameWriter writer;
+
+ public void configure(Map<String, String> configuration) {
+ // no-op
+ }
+
+ public void initialize(IHyracksTaskContext ctx, IFrameWriter writer)
+ throws HyracksDataException {
+ this.appender = new FrameTupleAppender();
+ this.frame = new VSizeFrame(ctx);
+ this.writer = writer;
+ appender.reset(frame, true);
+ }
+
+ public void addTuple(ArrayTupleBuilder tb) throws HyracksDataException {
+ boolean success = appender.append(tb.getFieldEndOffsets(),
+ tb.getByteArray(), 0, tb.getSize());
+ if (!success) {
+ FrameUtils.flushFrame(frame.getBuffer(), writer);
+ appender.reset(frame, true);
+ success = appender.append(tb.getFieldEndOffsets(),
+ tb.getByteArray(), 0, tb.getSize());
+ if (!success) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+ public void close() throws HyracksDataException {
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame.getBuffer(), writer);
+ }
+
+ }
+
+ @Override
+ public TupleForwardPolicyType getType() {
+ return TupleForwardPolicyType.FRAME_FULL;
+ }
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/NtDelimitedDataTupleParserFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/NtDelimitedDataTupleParserFactory.java
deleted file mode 100644
index 0e85fd2..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/NtDelimitedDataTupleParserFactory.java
+++ /dev/null
@@ -1,52 +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.runtime.operators.file;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-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;
-
-/**
- * A tuple parser factory for creating a tuple parser capable of parsing
- * delimited data.
- */
-public class NtDelimitedDataTupleParserFactory implements ITupleParserFactory {
- private static final long serialVersionUID = 1L;
- protected final ARecordType recordType;
- protected IValueParserFactory[] valueParserFactories;
- protected final char fieldDelimiter;
- // quote is used to enclose a string if it includes delimiter(s) in it.
- protected final char quote;
- // whether delimited text file has a header (which should be ignored)
- protected final boolean hasHeader;
-
- public NtDelimitedDataTupleParserFactory(ARecordType recordType, IValueParserFactory[] valueParserFactories,
- char fieldDelimiter, char quote, boolean hasHeader) {
- this.recordType = recordType;
- this.valueParserFactories = valueParserFactories;
- this.fieldDelimiter = fieldDelimiter;
- this.quote = quote;
- this.hasHeader = hasHeader;
- }
-
- @Override
- public ITupleParser createTupleParser(final IHyracksTaskContext ctx) throws HyracksDataException {
- return new DelimitedDataTupleParser(ctx, recordType, valueParserFactories, fieldDelimiter, quote,
- hasHeader);
- }
-}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/RateContolledParserPolicy.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/RateContolledParserPolicy.java
new file mode 100644
index 0000000..3711f94
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/RateContolledParserPolicy.java
@@ -0,0 +1,79 @@
+package edu.uci.ics.asterix.runtime.operators.file;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.parse.ITupleParserPolicy;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+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;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+
+public class RateContolledParserPolicy implements ITupleParserPolicy {
+
+ protected FrameTupleAppender appender;
+ protected IFrame frame;
+ private IFrameWriter writer;
+ private long interTupleInterval;
+ private boolean delayConfigured;
+
+ public static final String INTER_TUPLE_INTERVAL = "tuple-interval";
+
+ public RateContolledParserPolicy() {
+
+ }
+
+ public TupleParserPolicy getType() {
+ return ITupleParserPolicy.TupleParserPolicy.FRAME_FULL;
+ }
+
+
+ @Override
+ public void addTuple(ArrayTupleBuilder tb) throws HyracksDataException {
+ if (delayConfigured) {
+ try {
+ Thread.sleep(interTupleInterval);
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ boolean success = appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
+ if (!success) {
+ FrameUtils.flushFrame(frame.getBuffer(), writer);
+ appender.reset(frame, true);
+ success = appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
+ if (!success) {
+ throw new IllegalStateException();
+ }
+ }
+ appender.reset(frame, true);
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame.getBuffer(), writer);
+ }
+ }
+
+ @Override
+ public void configure(Map<String, String> configuration) throws HyracksDataException {
+ String propValue = configuration.get(INTER_TUPLE_INTERVAL);
+ if (propValue != null) {
+ interTupleInterval = Long.parseLong(propValue);
+ } else {
+ interTupleInterval = 0;
+ }
+ delayConfigured = interTupleInterval != 0;
+
+ }
+
+ @Override
+ public void initialize(IHyracksTaskContext ctx, IFrameWriter writer) throws HyracksDataException {
+ this.appender = new FrameTupleAppender();
+ this.frame = new VSizeFrame(ctx);
+ }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/RateControlledTupleForwardPolicy.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/RateControlledTupleForwardPolicy.java
new file mode 100644
index 0000000..896ac73
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/RateControlledTupleForwardPolicy.java
@@ -0,0 +1,70 @@
+package edu.uci.ics.asterix.runtime.operators.file;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.common.parse.ITupleForwardPolicy;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+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;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+
+public class RateControlledTupleForwardPolicy implements ITupleForwardPolicy {
+
+ private FrameTupleAppender appender;
+ private IFrame frame;
+ private IFrameWriter writer;
+ private long interTupleInterval;
+ private boolean delayConfigured;
+
+ public static final String INTER_TUPLE_INTERVAL = "tuple-interval";
+
+ public void configure(Map<String, String> configuration) {
+ String propValue = configuration.get(INTER_TUPLE_INTERVAL);
+ if (propValue != null) {
+ interTupleInterval = Long.parseLong(propValue);
+ }
+ delayConfigured = interTupleInterval != 0;
+ }
+
+ public void initialize(IHyracksTaskContext ctx, IFrameWriter writer) throws HyracksDataException {
+ this.appender = new FrameTupleAppender();
+ this.frame = new VSizeFrame(ctx);
+ this.writer = writer;
+ appender.reset(frame, true);
+ }
+
+ public void addTuple(ArrayTupleBuilder tb) throws HyracksDataException {
+ if (delayConfigured) {
+ try {
+ Thread.sleep(interTupleInterval);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ boolean success = appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
+ if (!success) {
+ FrameUtils.flushFrame(frame.getBuffer(), writer);
+ appender.reset(frame, true);
+ success = appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
+ if (!success) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+ public void close() throws HyracksDataException {
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame.getBuffer(), writer);
+ }
+
+ }
+
+ @Override
+ public TupleForwardPolicyType getType() {
+ return TupleForwardPolicyType.RATE_CONTROLLED;
+ }
+}
\ No newline at end of file
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
index ce22887..b8243a3 100644
--- 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
@@ -1,5 +1,5 @@
/*
-x * Copyright 2009-2013 by The Regents of the University of California
+ * 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
@@ -21,21 +21,18 @@
import java.util.List;
import java.util.Random;
+import edu.uci.ics.asterix.external.util.Datatypes;
+
public class DataGenerator {
private RandomDateGenerator randDateGen;
-
private RandomNameGenerator randNameGen;
-
private RandomMessageGenerator randMessageGen;
-
private RandomLocationGenerator randLocationGen;
-
private Random random = new Random();
-
private TwitterUser twUser = new TwitterUser();
-
private TweetMessage twMessage = new TweetMessage();
+ private static final String DEFAULT_COUNTRY = "US";
public DataGenerator(InitializationInfo info) {
initialize(info);
@@ -44,28 +41,32 @@
public class TweetMessageIterator implements Iterator<TweetMessage> {
private final int duration;
- private final GULongIDGenerator idGen;
private long startTime = 0;
+ private int tweetId;
- public TweetMessageIterator(int duration, GULongIDGenerator idGen) {
+ public TweetMessageIterator(int duration) {
this.duration = duration;
- this.idGen = idGen;
this.startTime = System.currentTimeMillis();
}
@Override
public boolean hasNext() {
+ if (duration == TweetGenerator.INFINITY) {
+ return true;
+ }
return System.currentTimeMillis() - startTime <= duration * 1000;
}
@Override
public TweetMessage next() {
+ tweetId++;
TweetMessage msg = null;
getTwitterUser(null);
Message message = randMessageGen.getNextRandomMessage();
Point location = randLocationGen.getRandomPoint();
DateTime sendTime = randDateGen.getNextRandomDatetime();
- twMessage.reset(idGen.getNextULong(), twUser, location, sendTime, message.getReferredTopics(), message);
+ twMessage.reset(tweetId, twUser, location.getLatitude(), location.getLongitude(), sendTime.toString(),
+ message, DEFAULT_COUNTRY);
msg = twMessage;
return msg;
}
@@ -73,6 +74,7 @@
@Override
public void remove() {
// TODO Auto-generated method stub
+
}
}
@@ -218,8 +220,7 @@
public String toString() {
StringBuilder builder = new StringBuilder();
- builder.append("datetime");
- builder.append("(\"");
+ builder.append("\"");
builder.append(super.getYear());
builder.append("-");
builder.append(super.getMonth() < 10 ? "0" + super.getMonth() : super.getMonth());
@@ -227,7 +228,7 @@
builder.append(super.getDay() < 10 ? "0" + super.getDay() : super.getDay());
builder.append("T");
builder.append(hour + ":" + min + ":" + sec);
- builder.append("\")");
+ builder.append("\"");
return builder.toString();
}
}
@@ -475,78 +476,112 @@
public static class TweetMessage {
- private long tweetid;
+ private static final String[] DEFAULT_FIELDS = new String[] { TweetFields.TWEETID, TweetFields.USER,
+ TweetFields.LATITUDE, TweetFields.LONGITUDE, TweetFields.MESSAGE_TEXT, TweetFields.CREATED_AT,
+ TweetFields.COUNTRY };
+
+ private int id;
private TwitterUser user;
- private Point senderLocation;
- private DateTime sendTime;
- private List<String> referredTopics;
+ private double latitude;
+ private double longitude;
+ private String created_at;
private Message messageText;
+ private String country;
+
+ public static final class TweetFields {
+ public static final String TWEETID = "id";
+ public static final String USER = "user";
+ public static final String LATITUDE = "latitude";
+ public static final String LONGITUDE = "longitude";
+ public static final String MESSAGE_TEXT = "message_text";
+ public static final String CREATED_AT = "created_at";
+ public static final String COUNTRY = "country";
+
+ }
public TweetMessage() {
}
- public TweetMessage(long tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
- List<String> referredTopics, Message messageText) {
- this.tweetid = tweetid;
+ public TweetMessage(int tweetid, TwitterUser user, double latitude, double longitude, String created_at,
+ Message messageText, String country) {
+ this.id = tweetid;
this.user = user;
- this.senderLocation = senderLocation;
- this.sendTime = sendTime;
- this.referredTopics = referredTopics;
+ this.latitude = latitude;
+ this.longitude = longitude;
+ this.created_at = created_at;
this.messageText = messageText;
+ this.country = country;
}
- public void reset(long tweetid, TwitterUser user, Point senderLocation, DateTime sendTime,
- List<String> referredTopics, Message messageText) {
- this.tweetid = tweetid;
+ public void reset(int tweetid, TwitterUser user, double latitude, double longitude, String created_at,
+ Message messageText, String country) {
+ this.id = tweetid;
this.user = user;
- this.senderLocation = senderLocation;
- this.sendTime = sendTime;
- this.referredTopics = referredTopics;
+ this.latitude = latitude;
+ this.longitude = longitude;
+ this.created_at = created_at;
this.messageText = messageText;
+ this.country = country;
}
- public String toString() {
+ public String getAdmEquivalent(String[] fields) {
+ if (fields == null) {
+ fields = DEFAULT_FIELDS;
+ }
StringBuilder builder = new StringBuilder();
builder.append("{");
- builder.append("\"tweetid\":");
- builder.append("int64(\"" + 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 + "\"");
+ for (String field : fields) {
+ switch (field) {
+ case Datatypes.Tweet.ID:
+ appendFieldName(builder, Datatypes.Tweet.ID);
+ builder.append("int64(\"" + id + "\")");
+ break;
+ case Datatypes.Tweet.USER:
+ appendFieldName(builder, Datatypes.Tweet.USER);
+ builder.append(user);
+ break;
+ case Datatypes.Tweet.LATITUDE:
+ appendFieldName(builder, Datatypes.Tweet.LATITUDE);
+ builder.append(latitude);
+ break;
+ case Datatypes.Tweet.LONGITUDE:
+ appendFieldName(builder, Datatypes.Tweet.LONGITUDE);
+ builder.append(longitude);
+ break;
+ case Datatypes.Tweet.MESSAGE:
+ appendFieldName(builder, Datatypes.Tweet.MESSAGE);
+ builder.append("\"");
+ for (int i = 0; i < messageText.getLength(); i++) {
+ builder.append(messageText.charAt(i));
+ }
+ builder.append("\"");
+ break;
+ case Datatypes.Tweet.CREATED_AT:
+ appendFieldName(builder, Datatypes.Tweet.CREATED_AT);
+ builder.append(created_at);
+ break;
+ case Datatypes.Tweet.COUNTRY:
+ appendFieldName(builder, Datatypes.Tweet.COUNTRY);
+ builder.append("\"" + country + "\"");
+ break;
+ }
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.deleteCharAt(builder.length() - 1);
builder.append("}");
- return new String(builder);
+ return builder.toString();
}
- public long getTweetid() {
- return tweetid;
+ private void appendFieldName(StringBuilder builder, String fieldName) {
+ builder.append("\"" + fieldName + "\":");
}
- public void setTweetid(long tweetid) {
- this.tweetid = tweetid;
+ public int getTweetid() {
+ return id;
+ }
+
+ public void setTweetid(int tweetid) {
+ this.id = tweetid;
}
public TwitterUser getUser() {
@@ -557,28 +592,12 @@
this.user = user;
}
- public Point getSenderLocation() {
- return senderLocation;
+ public double getLatitude() {
+ return latitude;
}
- 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 String getSendTime() {
+ return created_at;
}
public Message getMessageText() {
@@ -589,6 +608,10 @@
this.messageText = messageText;
}
+ public String getCountry() {
+ return country;
+ }
+
}
public static class TwitterUser {
@@ -643,13 +666,13 @@
public String toString() {
StringBuilder builder = new StringBuilder();
builder.append("{");
- builder.append("\"screen-name\":" + "\"" + screenName + "\"");
+ builder.append("\"screen_name\":" + "\"" + screenName + "\"");
builder.append(",");
- builder.append("\"lang\":" + "\"" + lang + "\"");
+ builder.append("\"language\":" + "\"" + lang + "\"");
builder.append(",");
builder.append("\"friends_count\":" + friendsCount);
builder.append(",");
- builder.append("\"statuses_count\":" + statusesCount);
+ builder.append("\"status_count\":" + statusesCount);
builder.append(",");
builder.append("\"name\":" + "\"" + name + "\"");
builder.append(",");
@@ -1158,5 +1181,4 @@
"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" };
-
}
\ No newline at end of file
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
index 20b9be1..993d9c9 100644
--- 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
@@ -7,8 +7,8 @@
import java.util.logging.Level;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.api.IFeedAdapter;
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;
@@ -18,12 +18,14 @@
private static final long serialVersionUID = 1L;
+ private final int port;
private SocketFeedServer socketFeedServer;
- public GenericSocketFeedAdapter(ITupleParserFactory parserFactory, ARecordType outputtype, int port,
- IHyracksTaskContext ctx) throws AsterixException, IOException {
- super(parserFactory, outputtype, ctx);
- this.socketFeedServer = new SocketFeedServer(outputtype, port);
+ public GenericSocketFeedAdapter(ITupleParserFactory parserFactory, ARecordType outputType, int port,
+ IHyracksTaskContext ctx, int partition) throws AsterixException, IOException {
+ super(parserFactory, outputType, ctx, partition);
+ this.port = port;
+ this.socketFeedServer = new SocketFeedServer(outputType, port);
}
@Override
@@ -90,4 +92,14 @@
return DataExchangeMode.PUSH;
}
+ @Override
+ public boolean handleException(Exception e) {
+ try {
+ this.socketFeedServer = new SocketFeedServer((ARecordType) sourceDatatype, port);
+ return true;
+ } catch (Exception re) {
+ return false;
+ }
+ }
+
}
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
index 17d989b..37d5141 100644
--- 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
@@ -23,12 +23,14 @@
import org.apache.commons.lang3.StringUtils;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
import edu.uci.ics.asterix.external.adapter.factory.StreamBasedAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapterFactory;
import edu.uci.ics.asterix.metadata.entities.ExternalFile;
-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.util.AsterixRuntimeUtil;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -40,7 +42,7 @@
* adapter listens at a port for receiving data (from external world).
* Data received is transformed into Asterix Data Format (ADM).
*/
-public class GenericSocketFeedAdapterFactory extends StreamBasedAdapterFactory implements IGenericAdapterFactory {
+public class GenericSocketFeedAdapterFactory extends StreamBasedAdapterFactory implements IFeedAdapterFactory {
private static final long serialVersionUID = 1L;
@@ -65,11 +67,6 @@
}
@Override
- public AdapterType getAdapterType() {
- return AdapterType.GENERIC;
- }
-
- @Override
public SupportedOperation getSupportedOperations() {
return SupportedOperation.READ;
}
@@ -81,9 +78,9 @@
@Override
public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
this.configuration = configuration;
- outputType = (ARecordType) outputType;
- this.configureFormat(outputType);
this.configureSockets(configuration);
+ this.configureFormat(outputType);
+ this.outputType = (ARecordType) outputType;
}
@Override
@@ -98,7 +95,7 @@
@Override
public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
Pair<String, Integer> socket = sockets.get(partition);
- return new GenericSocketFeedAdapter(parserFactory, outputType, socket.second, ctx);
+ return new GenericSocketFeedAdapter(parserFactory, outputType, socket.second, ctx, partition);
}
private void configureSockets(Map<String, String> configuration) throws Exception {
@@ -117,8 +114,8 @@
Random random = new Random();
for (String socket : socketsArray) {
String[] socketTokens = socket.split(":");
- String host = socketTokens[0];
- int port = Integer.parseInt(socketTokens[1]);
+ String host = socketTokens[0].trim();
+ int port = Integer.parseInt(socketTokens[1].trim());
Pair<String, Integer> p = null;
switch (mode) {
case IP:
@@ -148,6 +145,23 @@
}
@Override
+ public ARecordType getAdapterOutputType() {
+ return outputType;
+ }
+
+ @Override
+ public InputDataFormat getInputDataFormat() {
+ return InputDataFormat.UNKNOWN;
+ }
+
+ public boolean isRecordTrackingEnabled() {
+ return false;
+ }
+
+ public IIntakeProgressTracker createIntakeProgressTracker() {
+ throw new UnsupportedOperationException("Tracking of ingested records not enabled");
+ }
+
public void setFiles(List<ExternalFile> files) throws AlgebricksException {
throw new AlgebricksException("files access not supported for this adapter");
}
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 40342cd..e18ba42 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,8 +18,8 @@
import java.io.InputStream;
import java.util.Map;
+import edu.uci.ics.asterix.common.feeds.api.IFeedAdapter;
import edu.uci.ics.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
import edu.uci.ics.asterix.om.types.ARecordType;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
@@ -49,7 +49,7 @@
@Override
public void stop() {
- ((RateControlledTupleParser) tupleParser).stop();
+ // ((RateControlledTupleParser) tupleParser).stop();
}
@Override
@@ -57,4 +57,9 @@
return DataExchangeMode.PULL;
}
+ @Override
+ public boolean handleException(Exception e) {
+ return false;
+ }
+
}
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 d3f13f1..2a18133 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,34 +14,21 @@
*/
package edu.uci.ics.asterix.tools.external.data;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.List;
import java.util.Map;
import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
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.metadata.entities.ExternalFile;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
-import edu.uci.ics.asterix.metadata.feeds.IGenericAdapterFactory;
+import edu.uci.ics.asterix.metadata.external.IAdapterFactory;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapterFactory;
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.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
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.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.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
@@ -50,7 +37,7 @@
* source file has been ingested.
*/
public class RateControlledFileSystemBasedAdapterFactory extends StreamBasedAdapterFactory implements
- IGenericAdapterFactory {
+ IFeedAdapterFactory {
private static final long serialVersionUID = 1L;
public static final String KEY_FILE_SYSTEM = "fs";
@@ -59,9 +46,8 @@
public static final String KEY_PATH = "path";
public static final String KEY_FORMAT = "format";
- private IGenericAdapterFactory adapterFactory;
+ private IAdapterFactory adapterFactory;
private String format;
- private Map<String, String> configuration;
private ARecordType atype;
@Override
@@ -79,8 +65,8 @@
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(IGenericAdapterFactory.KEY_TYPE_NAME) == null) {
- throw new Exception("Record type not specified (output-type-name=?)");
+ if (configuration.get(IAdapterFactory.KEY_TYPE_NAME) == null) {
+ throw new Exception("Record type not specified (type-name=?)");
}
if (configuration.get(KEY_PATH) == null) {
throw new Exception("File path not specified (path=?)");
@@ -91,17 +77,12 @@
}
@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 {
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
this.configuration = configuration;
checkRequiredArgs(configuration);
String fileSystem = (String) configuration.get(KEY_FILE_SYSTEM);
@@ -113,12 +94,11 @@
} else {
throw new AsterixException("Unsupported file system type " + fileSystem);
}
+ this.atype = outputType;
format = configuration.get(KEY_FORMAT);
- adapterFactory = (IGenericAdapterFactory) Class.forName(adapterFactoryClass).newInstance();
- adapterFactory.configure(configuration, recordType);
-
- atype = (ARecordType) recordType;
- configureFormat();
+ adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClass).newInstance();
+ adapterFactory.configure(configuration, outputType);
+ configureFormat(outputType);
}
@Override
@@ -126,155 +106,22 @@
return adapterFactory.getPartitionConstraint();
}
- private void configureFormat() throws AsterixException {
- switch (format) {
- case FORMAT_ADM:
- parserFactory = new RateControlledTupleParserFactory(atype, configuration);
- break;
-
- case FORMAT_DELIMITED_TEXT:
- char delimiter = StreamBasedAdapterFactory.getDelimiter(configuration);
- char quote = StreamBasedAdapterFactory.getQuote(configuration, delimiter);
- boolean hasHeader = StreamBasedAdapterFactory.getHasHeader(configuration);
- IValueParserFactory[] valueParserFactories = getValueParserFactories(atype);
- parserFactory = new RateControlledTupleParserFactory(atype, valueParserFactories, delimiter, quote,
- hasHeader, 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;
+ @Override
+ public ARecordType getAdapterOutputType() {
+ return atype;
}
@Override
- public void setFiles(List<ExternalFile> files) throws AlgebricksException {
- throw new AlgebricksException("can't set files for this Adapter");
+ public InputDataFormat getInputDataFormat() {
+ return InputDataFormat.UNKNOWN;
}
-}
-
-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 char quote;
- private boolean hasHeader;
- private final ParserType parserType;
-
- public enum ParserType {
- ADM,
- DELIMITED_DATA
+ public boolean isRecordTrackingEnabled() {
+ return false;
}
- public RateControlledTupleParserFactory(ARecordType recordType, IValueParserFactory[] valueParserFactories,
- char fieldDelimiter, char quote, boolean hasHeader, Map<String, String> configuration) {
- this.recordType = recordType;
- this.valueParserFactories = valueParserFactories;
- this.delimiter = fieldDelimiter;
- this.quote = quote;
- this.hasHeader = hasHeader;
- this.configuration = configuration;
- this.parserType = ParserType.DELIMITED_DATA;
+ public IIntakeProgressTracker createIntakeProgressTracker() {
+ throw new UnsupportedOperationException("Tracking of ingested records not enabled");
}
- 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, quote, hasHeader);
- 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 {
-
- 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);
- }
- appender.flush(writer, true);
- } 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/SocketClientAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SocketClientAdapter.java
index 4cbc4f1..dcdf9c1 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SocketClientAdapter.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SocketClientAdapter.java
@@ -21,7 +21,7 @@
import java.util.logging.Level;
import java.util.logging.Logger;
-import edu.uci.ics.asterix.metadata.feeds.IFeedAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IFeedAdapter;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -33,6 +33,8 @@
private static final String LOCALHOST = "127.0.0.1";
+ private static final long RECONNECT_PERIOD = 2000;
+
private final String localFile;
private final int port;
@@ -49,7 +51,7 @@
@Override
public void start(int partition, IFrameWriter writer) throws Exception {
- Socket socket = new Socket(LOCALHOST, port);
+ Socket socket = waitForReceiver();
OutputStream os = socket.getOutputStream();
FileInputStream fin = new FileInputStream(new File(localFile));
byte[] chunk = new byte[1024];
@@ -63,13 +65,31 @@
break;
}
}
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Finished streaming file " + localFile + "to port [" + port + "]");
+ }
+
} finally {
socket.close();
fin.close();
}
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Finished streaming file " + localFile + "to port [" + port + "]");
+
+ }
+
+ private Socket waitForReceiver() throws Exception {
+ Socket socket = null;
+ while (socket == null) {
+ try {
+ socket = new Socket(LOCALHOST, port);
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Receiver not ready, would wait for " + (RECONNECT_PERIOD / 1000)
+ + " seconds before reconnecting");
+ }
+ Thread.sleep(RECONNECT_PERIOD);
+ }
}
+ return socket;
}
@Override
@@ -82,4 +102,9 @@
continueStreaming = false;
}
+ @Override
+ public boolean handleException(Exception e) {
+ return false;
+ }
+
}
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SocketClientAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SocketClientAdapterFactory.java
index 24a89dd..ef18197 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SocketClientAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/SocketClientAdapterFactory.java
@@ -16,23 +16,20 @@
import java.util.Map;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.metadata.feeds.IDatasourceAdapter;
-import edu.uci.ics.asterix.metadata.feeds.ITypedAdapterFactory;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
+import edu.uci.ics.asterix.metadata.feeds.IFeedAdapterFactory;
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.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-public class SocketClientAdapterFactory implements ITypedAdapterFactory {
+public class SocketClientAdapterFactory implements IFeedAdapterFactory {
private static final long serialVersionUID = 1L;
- private static final ARecordType outputType = initOutputType();
+ private ARecordType outputType;
private GenericSocketFeedAdapterFactory genericSocketAdapterFactory;
@@ -41,32 +38,21 @@
public static final String KEY_FILE_SPLITS = "file_splits";
@Override
- public SupportedOperation getSupportedOperations() {
- return SupportedOperation.READ;
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ this.outputType = outputType;
+ String fileSplitsValue = configuration.get(KEY_FILE_SPLITS);
+ if (fileSplitsValue == null) {
+ throw new IllegalArgumentException(
+ "File splits not specified. File split is specified as a comma separated list of paths");
+ }
+ fileSplits = fileSplitsValue.trim().split(",");
+ genericSocketAdapterFactory = new GenericSocketFeedAdapterFactory();
+ genericSocketAdapterFactory.configure(configuration, 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.AINT64, userRecordType, BuiltinType.APOINT,
- BuiltinType.ADATETIME, unorderedListType, BuiltinType.ASTRING };
- outputType = new ARecordType("TweetMessageType", fieldNames, fieldTypes, false);
-
- } catch (AsterixException | HyracksDataException e) {
- throw new IllegalStateException("Unable to initialize output type");
- }
- return outputType;
+ @Override
+ public SupportedOperation getSupportedOperations() {
+ return SupportedOperation.READ;
}
@Override
@@ -75,11 +61,6 @@
}
@Override
- public AdapterType getAdapterType() {
- return AdapterType.TYPED;
- }
-
- @Override
public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
return genericSocketAdapterFactory.getPartitionConstraint();
}
@@ -96,14 +77,13 @@
}
@Override
- public void configure(Map<String, String> configuration) throws Exception {
- String fileSplitsValue = configuration.get(KEY_FILE_SPLITS);
- if (fileSplitsValue == null) {
- throw new IllegalArgumentException(
- "File splits not specified. File split is specified as a comma separated list of paths");
- }
- fileSplits = fileSplitsValue.trim().split(",");
- genericSocketAdapterFactory = new GenericSocketFeedAdapterFactory();
- genericSocketAdapterFactory.configure(configuration, outputType);
+ public boolean isRecordTrackingEnabled() {
+ return false;
}
+
+ @Override
+ public IIntakeProgressTracker createIntakeProgressTracker() {
+ return null;
+ }
+
}
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
index 8f252e6..961f497 100644
--- 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
@@ -1,5 +1,5 @@
/*
-x * Copyright 2009-2013 by The Regents of the University of California
+ * 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
@@ -17,6 +17,8 @@
import java.io.IOException;
import java.io.OutputStream;
import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
import java.util.Map;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -31,45 +33,38 @@
public static final String KEY_DURATION = "duration";
public static final String KEY_TPS = "tps";
- public static final String KEY_GUID_SEED = "guid-seed";
+ public static final String KEY_VERBOSE = "verbose";
+ public static final String KEY_FIELDS = "fields";
+ public static final int INFINITY = 0;
- 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 static final int DEFAULT_DURATION = 60; //seconds
- private static final int DEFAULT_GUID_SEED = 0;
+ private static final int DEFAULT_DURATION = INFINITY;
private int duration;
private TweetMessageIterator tweetIterator = null;
private int partition;
- private int tweetCount = 0;
+ private long tweetCount = 0;
private int frameTweetCount = 0;
private int numFlushedTweets = 0;
- private OutputStream os;
private DataGenerator dataGenerator = null;
private ByteBuffer outputBuffer = ByteBuffer.allocate(32 * 1024);
- private GULongIDGenerator uidGenerator;
+ private String[] fields;
+ private final List<OutputStream> subscribers;
+ private final Object lock = new Object();
+ private final List<OutputStream> subscribersForRemoval = new ArrayList<OutputStream>();
- public int getTweetCount() {
- return tweetCount;
- }
-
- public TweetGenerator(Map<String, String> configuration, int partition, String format, OutputStream os)
- throws Exception {
+ public TweetGenerator(Map<String, String> configuration, int partition) throws Exception {
this.partition = partition;
String value = configuration.get(KEY_DURATION);
this.duration = value != null ? Integer.parseInt(value) : DEFAULT_DURATION;
- int guidSeed = configuration.get(KEY_GUID_SEED) != null ? Integer.parseInt(configuration.get(KEY_GUID_SEED))
- : DEFAULT_GUID_SEED;
- uidGenerator = new GULongIDGenerator(partition, (byte) (guidSeed));
dataGenerator = new DataGenerator(new InitializationInfo());
- tweetIterator = dataGenerator.new TweetMessageIterator(duration, uidGenerator);
- this.os = os;
+ tweetIterator = dataGenerator.new TweetMessageIterator(duration);
+ this.fields = configuration.get(KEY_FIELDS) != null ? configuration.get(KEY_FIELDS).split(",") : null;
+ this.subscribers = new ArrayList<OutputStream>();
}
private void writeTweetString(TweetMessage tweetMessage) throws IOException {
- String tweet = tweetMessage.toString() + "\n";
+ String tweet = tweetMessage.getAdmEquivalent(fields) + "\n";
+ System.out.println(tweet);
tweetCount++;
byte[] b = tweet.getBytes();
if (outputBuffer.position() + b.length > outputBuffer.limit()) {
@@ -83,18 +78,26 @@
frameTweetCount++;
}
- public int getNumFlushedTweets() {
- return numFlushedTweets;
- }
-
private void flush() throws IOException {
outputBuffer.flip();
- os.write(outputBuffer.array(), 0, outputBuffer.limit());
+ synchronized (lock) {
+ for (OutputStream os : subscribers) {
+ try {
+ os.write(outputBuffer.array(), 0, outputBuffer.limit());
+ } catch (Exception e) {
+ subscribersForRemoval.add(os);
+ }
+ }
+ if (!subscribersForRemoval.isEmpty()) {
+ subscribers.removeAll(subscribersForRemoval);
+ subscribersForRemoval.clear();
+ }
+ }
outputBuffer.position(0);
outputBuffer.limit(32 * 1024);
}
- public boolean setNextRecordBatch(int numTweetsInBatch) throws Exception {
+ public boolean generateNextBatch(int numTweets) throws Exception {
boolean moreData = tweetIterator.hasNext();
if (!moreData) {
if (outputBuffer.position() > 0) {
@@ -106,11 +109,44 @@
return false;
} else {
int count = 0;
- while (count < numTweetsInBatch) {
+ while (count < numTweets) {
writeTweetString(tweetIterator.next());
count++;
}
return true;
}
}
+
+ public int getNumFlushedTweets() {
+ return numFlushedTweets;
+ }
+
+ public void registerSubscriber(OutputStream os) {
+ synchronized (lock) {
+ subscribers.add(os);
+ }
+ }
+
+ public void deregisterSubscribers(OutputStream os) {
+ synchronized (lock) {
+ subscribers.remove(os);
+ }
+ }
+
+ public void close() throws IOException {
+ synchronized (lock) {
+ for (OutputStream os : subscribers) {
+ os.close();
+ }
+ }
+ }
+
+ public boolean isSubscribed() {
+ return !subscribers.isEmpty();
+ }
+
+ public long getTweetCount() {
+ return tweetCount;
+ }
+
}
\ No newline at end of file
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
index e0691a6..b8ef691 100644
--- 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
@@ -1,5 +1,5 @@
/*
-x * Copyright 2009-2013 by The Regents of the University of California
+ * 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
@@ -19,23 +19,24 @@
import java.io.OutputStream;
import java.io.PipedInputStream;
import java.io.PipedOutputStream;
+import java.util.ArrayList;
+import java.util.List;
import java.util.Map;
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.common.feeds.api.IFeedAdapter;
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
- *
- * @author ramang
+ * A simulator of the Twitter Firehose. Generates meaningful tweets
+ * at a configurable rate
*/
public class TwitterFirehoseFeedAdapter extends StreamBasedAdapter implements IFeedAdapter {
@@ -52,8 +53,8 @@
private final TwitterServer twitterServer;
public TwitterFirehoseFeedAdapter(Map<String, String> configuration, ITupleParserFactory parserFactory,
- ARecordType outputtype, int partition, IHyracksTaskContext ctx) throws Exception {
- super(parserFactory, outputtype, ctx);
+ ARecordType outputtype, IHyracksTaskContext ctx, int partition) throws Exception {
+ super(parserFactory, outputtype, ctx, partition);
this.twitterServer = new TwitterServer(configuration, partition, outputtype, outputStream, executorService);
}
@@ -68,7 +69,7 @@
return inputStream;
}
- public static class TwitterServer {
+ private static class TwitterServer {
private final DataProvider dataProvider;
private final ExecutorService executorService;
@@ -88,7 +89,7 @@
}
- public static class DataProvider implements Runnable {
+ private static class DataProvider implements Runnable {
public static final String KEY_MODE = "mode";
@@ -105,8 +106,8 @@
public DataProvider(Map<String, String> configuration, ARecordType outputtype, int partition, OutputStream os)
throws Exception {
- this.tweetGenerator = new TweetGenerator(configuration, partition, TweetGenerator.OUTPUT_FORMAT_ADM_STRING,
- os);
+ this.tweetGenerator = new TweetGenerator(configuration, partition);
+ this.tweetGenerator.registerSubscriber(os);
this.os = os;
mode = configuration.get(KEY_MODE) != null ? Mode.valueOf(configuration.get(KEY_MODE).toUpperCase())
: Mode.AGGRESSIVE;
@@ -130,30 +131,29 @@
long startBatch;
long endBatch;
- try {
- while (moreData && continuePush) {
- switch (mode) {
- case AGGRESSIVE:
- moreData = tweetGenerator.setNextRecordBatch(batchSize);
- break;
- case CONTROLLED:
- startBatch = System.currentTimeMillis();
- moreData = tweetGenerator.setNextRecordBatch(batchSize);
- endBatch = System.currentTimeMillis();
- if (endBatch - startBatch < 1000) {
- Thread.sleep(1000 - (endBatch - startBatch));
- } else {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Unable to reach the required tps of " + batchSize);
+ while (true) {
+ try {
+ while (moreData && continuePush) {
+ switch (mode) {
+ case AGGRESSIVE:
+ moreData = tweetGenerator.generateNextBatch(batchSize);
+ break;
+ case CONTROLLED:
+ startBatch = System.currentTimeMillis();
+ moreData = tweetGenerator.generateNextBatch(batchSize);
+ endBatch = System.currentTimeMillis();
+ if (endBatch - startBatch < 1000) {
+ Thread.sleep(1000 - (endBatch - startBatch));
}
- }
- break;
+ break;
+ }
}
- }
- os.close();
- } catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
- LOGGER.warning("Exception in adapter " + e.getMessage());
+ os.close();
+ break;
+ } catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
+ LOGGER.warning("Exception in adaptor " + e.getMessage());
+ }
}
}
}
@@ -174,4 +174,16 @@
return DataExchangeMode.PUSH;
}
+ @Override
+ public boolean handleException(Exception e) {
+ try {
+ twitterServer.stop();
+ } catch (Exception re) {
+ re.printStackTrace();
+ return false;
+ }
+ twitterServer.start();
+ return true;
+ }
+
}
\ No newline at end of file
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
index f2f730f..56976d2 100644
--- 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
@@ -18,41 +18,41 @@
import java.util.List;
import java.util.Map;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.feeds.api.IDatasourceAdapter;
+import edu.uci.ics.asterix.common.feeds.api.IIntakeProgressTracker;
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.metadata.feeds.IFeedAdapterFactory;
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.asterix.runtime.operators.file.AsterixTupleParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.AsterixTupleParserFactory.InputDataFormat;
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.exceptions.HyracksDataException;
/**
- * 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).
+ * 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 {
+public class TwitterFirehoseFeedAdapterFactory extends StreamBasedAdapterFactory implements IFeedAdapterFactory {
private static final long serialVersionUID = 1L;
- /*
- * Degree of parallelism for feed ingestion activity. Defaults to 1.
- * This builds up the count constraint for the ingestion operator.
- */
+ /**
+ * Degree of parallelism for feed ingestion activity. Defaults to 1. This
+ * determines 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.
- */
+ /**
+ * The absolute locations where ingestion operator instances will be placed.
+ **/
private static final String KEY_INGESTION_LOCATIONS = "ingestion-location";
- private static final ARecordType outputType = initOutputType();
+ private ARecordType outputType;
@Override
public String getName() {
@@ -60,20 +60,16 @@
}
@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);
+ public void configure(Map<String, String> configuration, ARecordType outputType) throws Exception {
+ configuration.put(AsterixTupleParserFactory.KEY_FORMAT, AsterixTupleParserFactory.FORMAT_ADM);
this.configuration = configuration;
- this.configureFormat(initOutputType());
+ this.outputType = outputType;
+ this.configureFormat(outputType);
}
@Override
@@ -100,7 +96,7 @@
@Override
public IDatasourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws Exception {
- return new TwitterFirehoseFeedAdapter(configuration, parserFactory, outputType, partition, ctx);
+ return new TwitterFirehoseFeedAdapter(configuration, parserFactory, outputType, ctx, partition);
}
@Override
@@ -108,27 +104,17 @@
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.AINT64, userRecordType, BuiltinType.APOINT,
- BuiltinType.ADATETIME, unorderedListType, BuiltinType.ASTRING };
- outputType = new ARecordType("TweetMessageType", fieldNames, fieldTypes, false);
-
- } catch (AsterixException | HyracksDataException e) {
- throw new IllegalStateException("Unable to initialize output type");
- }
- return outputType;
+ @Override
+ public InputDataFormat getInputDataFormat() {
+ return InputDataFormat.ADM;
}
+
+ public boolean isRecordTrackingEnabled() {
+ return false;
+ }
+
+ public IIntakeProgressTracker createIntakeProgressTracker() {
+ throw new UnsupportedOperationException("Tracking of ingested records not enabled");
+ }
+
}
\ No newline at end of file