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-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-app/src/test/resources/runtimets/results/feeds/feeds_09/feeds_09.1.adm b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_09/feeds_09.1.adm
new file mode 100644
index 0000000..3c13d5f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/feeds/feeds_09/feeds_09.1.adm
@@ -0,0 +1,2 @@
+[ 1
+ ]
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>