You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by xi...@apache.org on 2017/10/21 16:10:40 UTC

[1/3] asterixdb git commit: [ASTERIXDB-1983] Feed pipeline refactoring for SQL++

Repository: asterixdb
Updated Branches:
  refs/heads/master 799046dfa -> 7e76a0797


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index 692ba2e..a418cbf 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -299,8 +299,8 @@ public class ExternalDataUtils {
         return Boolean.parseBoolean(configuration.get(ExternalDataConstants.KEY_IS_CHANGE_FEED));
     }
 
-    public static boolean isUpsertFeed(Map<String, String> configuration) {
-        return Boolean.parseBoolean(configuration.get(ExternalDataConstants.KEY_IS_UPSERT_FEED));
+    public static boolean isInsertFeed(Map<String, String> configuration) {
+        return Boolean.parseBoolean(configuration.get(ExternalDataConstants.KEY_IS_INSERT_FEED));
     }
 
     public static int getNumberOfKeys(Map<String, String> configuration) throws AsterixException {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedConstants.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedConstants.java
index 9538711..f42c030 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/FeedConstants.java
@@ -18,8 +18,14 @@
  */
 package org.apache.asterix.external.util;
 
+import org.apache.asterix.common.functions.FunctionConstants;
+import org.apache.asterix.common.functions.FunctionSignature;
+
 public class FeedConstants {
 
+    public static final FunctionSignature FEED_COLLECT_FUN_SIGNATURE =
+            new FunctionSignature(FunctionConstants.ASTERIX_NS, "feed_collect", 6);
+
     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";
@@ -31,7 +37,6 @@ public class FeedConstants {
         public static final String INTAKE_TIMESTAMP = "intake-timestamp";
         public static final String COMPUTE_TIMESTAMP = "compute-timestamp";
         public static final String STORE_TIMESTAMP = "store-timestamp";
-
     }
 
     public static final class MessageConstants {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-lang-aql/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-aql/pom.xml b/asterixdb/asterix-lang-aql/pom.xml
index dd29e7c..8ed16ad 100644
--- a/asterixdb/asterix-lang-aql/pom.xml
+++ b/asterixdb/asterix-lang-aql/pom.xml
@@ -147,21 +147,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.asterix</groupId>
-      <artifactId>asterix-active</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.asterix</groupId>
       <artifactId>asterix-metadata</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.asterix</groupId>
-      <artifactId>asterix-external-data</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.asterix</groupId>
       <artifactId>asterix-common</artifactId>
       <version>${project.version}</version>
     </dependency>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/parser/FunctionParser.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/parser/FunctionParser.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/parser/FunctionParser.java
index 47a9580..098b447 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/parser/FunctionParser.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/parser/FunctionParser.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.Statement;
@@ -40,6 +41,10 @@ public class FunctionParser {
     }
 
     public FunctionDecl getFunctionDecl(Function function) throws CompilationException {
+        if (!function.getLanguage().equals(Function.LANGUAGE_AQL)) {
+            throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_FUNCTION_LANGUAGE,
+                    Function.LANGUAGE_AQL, function.getLanguage());
+        }
         String functionBody = function.getFunctionBody();
         List<String> params = function.getParams();
         List<VarIdentifier> varIdentifiers = new ArrayList<VarIdentifier>();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java
deleted file mode 100644
index f0539c6..0000000
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/statement/SubscribeFeedStatement.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.lang.aql.statement;
-
-import java.io.StringReader;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import org.apache.asterix.active.EntityId;
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.MetadataException;
-import org.apache.asterix.common.functions.FunctionSignature;
-import org.apache.asterix.external.feed.management.FeedConnectionRequest;
-import org.apache.asterix.external.feed.watch.FeedActivityDetails;
-import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.lang.aql.parser.AQLParserFactory;
-import org.apache.asterix.lang.common.base.IParser;
-import org.apache.asterix.lang.common.base.IParserFactory;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.statement.InsertStatement;
-import org.apache.asterix.lang.common.statement.Query;
-import org.apache.asterix.lang.common.util.FunctionUtil;
-import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-import org.apache.asterix.metadata.MetadataManager;
-import org.apache.asterix.metadata.MetadataTransactionContext;
-import org.apache.asterix.metadata.entities.Feed;
-import org.apache.asterix.metadata.entities.Function;
-import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
-
-/**
- * Represents the AQL statement for subscribing to a feed.
- * This AQL statement is private and may not be used by the end-user.
- */
-public class SubscribeFeedStatement implements Statement {
-
-    public static final String WAIT_FOR_COMPLETION = "wait-for-completion-feed";
-    private static final Integer INSERT_STATEMENT_POS = 3;
-    private static final Logger LOGGER = Logger.getLogger(SubscribeFeedStatement.class.getName());
-    private final int varCounter;
-    private final String[] locations;
-    private final FeedConnectionRequest connectionRequest;
-    private final IParserFactory parserFactory = new AQLParserFactory();
-    private Query query;
-
-    public SubscribeFeedStatement(String[] locations, FeedConnectionRequest subscriptionRequest) {
-        this.connectionRequest = subscriptionRequest;
-        this.varCounter = 0;
-        this.locations = locations;
-    }
-
-    public void initialize(MetadataTransactionContext mdTxnCtx) throws MetadataException {
-        this.query = new Query(false);
-        EntityId sourceFeedId = connectionRequest.getReceivingFeedId();
-        Feed subscriberFeed =
-                MetadataManager.INSTANCE.getFeed(mdTxnCtx, connectionRequest.getReceivingFeedId().getDataverse(),
-                        connectionRequest.getReceivingFeedId().getEntityName());
-        if (subscriberFeed == null) {
-            throw new IllegalStateException(" Subscriber feed " + subscriberFeed + " not found.");
-        }
-
-        String feedOutputType = getOutputType(mdTxnCtx);
-        StringBuilder builder = new StringBuilder();
-        builder.append("use dataverse " + sourceFeedId.getDataverse() + ";\n");
-        builder.append("set" + " " + FunctionUtil.IMPORT_PRIVATE_FUNCTIONS + " " + "'" + Boolean.TRUE + "'" + ";\n");
-        builder.append("set" + " " + FeedActivityDetails.FEED_POLICY_NAME + " " + "'" + connectionRequest.getPolicy()
-                + "'" + ";\n");
-
-        builder.append("insert into dataset " + connectionRequest.getTargetDataset() + " ");
-        builder.append(" (" + " for $x in feed-collect ('" + sourceFeedId.getDataverse() + "'" + "," + "'"
-                + sourceFeedId.getEntityName() + "'" + "," + "'"
-                + connectionRequest.getReceivingFeedId().getEntityName() + "'" + "," + "'"
-                + connectionRequest.getSubscriptionLocation().name() + "'" + "," + "'"
-                + connectionRequest.getTargetDataset() + "'" + "," + "'" + feedOutputType + "'" + ")");
-
-        List<FunctionSignature> functionsToApply = connectionRequest.getFunctionsToApply();
-        if ((functionsToApply != null) && functionsToApply.isEmpty()) {
-            builder.append(" return $x");
-        } else {
-            Function function;
-            String rValueName = "x";
-            String lValueName = "y";
-            int variableIndex = 0;
-            for (FunctionSignature appliedFunction : functionsToApply) {
-                function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, appliedFunction);
-                variableIndex++;
-                switch (function.getLanguage().toUpperCase()) {
-                    case Function.LANGUAGE_AQL:
-                        builder.append(" let " + "$" + lValueName + variableIndex + ":=" + function.getName() + "("
-                                + "$" + rValueName + ")");
-                        rValueName = lValueName + variableIndex;
-                        break;
-                    case Function.LANGUAGE_JAVA:
-                        builder.append(" let " + "$" + lValueName + variableIndex + ":=" + function.getName() + "("
-                                + "$" + rValueName + ")");
-                        rValueName = lValueName + variableIndex;
-                        break;
-                }
-                builder.append("\n");
-            }
-            builder.append("return $" + lValueName + variableIndex);
-        }
-        builder.append(")");
-        builder.append(";");
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Connect feed statement translated to\n" + builder.toString());
-        }
-        IParser parser = parserFactory.createParser(new StringReader(builder.toString()));
-
-        List<Statement> statements;
-        try {
-            statements = parser.parse();
-            query = ((InsertStatement) statements.get(INSERT_STATEMENT_POS)).getQuery();
-        } catch (CompilationException pe) {
-            throw new MetadataException(pe);
-        }
-
-    }
-
-    public Query getQuery() {
-        return query;
-    }
-
-    public int getVarCounter() {
-        return varCounter;
-    }
-
-    @Override
-    public byte getKind() {
-        return Statement.Kind.SUBSCRIBE_FEED;
-    }
-
-    public String getPolicy() {
-        return connectionRequest.getPolicy();
-    }
-
-    public FeedConnectionRequest getSubscriptionRequest() {
-        return connectionRequest;
-    }
-
-    @Override
-    public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
-        return null;
-    }
-
-    public String getDataverseName() {
-        return connectionRequest.getReceivingFeedId().getDataverse();
-    }
-
-    private String getOutputType(MetadataTransactionContext mdTxnCtx) throws MetadataException {
-        String outputType;
-        EntityId feedId = connectionRequest.getReceivingFeedId();
-        Feed feed = MetadataManager.INSTANCE.getFeed(mdTxnCtx, feedId.getDataverse(), feedId.getEntityName());
-        try {
-            outputType = FeedMetadataUtil
-                    .getOutputType(feed, feed.getAdapterConfiguration(), ExternalDataConstants.KEY_TYPE_NAME)
-                    .getTypeName();
-            return outputType;
-
-        } catch (MetadataException ae) {
-            throw new MetadataException(ae);
-        }
-    }
-
-    public String[] getLocations() {
-        return locations;
-    }
-
-    @Override
-    public byte getCategory() {
-        return Category.PROCEDURE;
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
index 351ac70..8d8ab73 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FunctionUtil.java
@@ -123,7 +123,8 @@ public class FunctionUtil {
                 throw new CompilationException(messageBuilder.toString());
             }
 
-            if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
+            if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)
+                    || function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_SQLPP)) {
                 FunctionDecl functionDecl = functionParser.getFunctionDecl(function);
                 if (functionDecl != null) {
                     if (functionDecls.contains(functionDecl)) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/FunctionParser.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/FunctionParser.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/FunctionParser.java
index 2b24ea1..4c0aa24 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/FunctionParser.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/FunctionParser.java
@@ -23,6 +23,7 @@ import java.io.StringReader;
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.Statement;
@@ -40,6 +41,10 @@ public class FunctionParser {
     }
 
     public FunctionDecl getFunctionDecl(Function function) throws CompilationException {
+        if (!function.getLanguage().equals(Function.LANGUAGE_SQLPP)) {
+            throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_FUNCTION_LANGUAGE,
+                    Function.LANGUAGE_SQLPP, function.getLanguage());
+        }
         String functionBody = function.getFunctionBody();
         List<String> params = function.getParams();
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index fa60bba..9156b0f 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -63,6 +63,7 @@ import org.apache.asterix.metadata.utils.IndexUtil;
 import org.apache.asterix.metadata.utils.InvertedIndexResourceFactoryProvider;
 import org.apache.asterix.metadata.utils.MetadataUtil;
 import org.apache.asterix.metadata.utils.RTreeResourceFactoryProvider;
+import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.utils.RecordUtil;
@@ -85,6 +86,10 @@ import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
 import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
@@ -289,8 +294,31 @@ public class Dataset implements IMetadataEntity<Dataset>, IDataset {
                 && Objects.equals(datasetName, otherDataset.datasetName);
     }
 
-    public boolean allow(ILogicalOperator topOp, byte operation) {//NOSONAR: this method is meant to be extended
-        return !hasMetaPart();
+    public boolean allow(ILogicalOperator topOp, byte operation) {
+        if (!hasMetaPart()) {
+            return true;
+        }
+        if (topOp.getInputs().get(0).getValue().getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+            return false;
+        }
+        ILogicalOperator op = topOp.getInputs().get(0).getValue();
+        while ((!op.getInputs().isEmpty())
+                && op.getInputs().get(0).getValue().getOperatorTag() != LogicalOperatorTag.UNNEST) {
+            op = op.getInputs().get(0).getValue();
+        }
+        if (op.getInputs().isEmpty()) {
+            return false;
+        }
+        UnnestOperator unnestOp = (UnnestOperator) op.getInputs().get(0).getValue();
+        if (unnestOp.getExpressionRef().getValue().getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression functionCall =
+                (AbstractFunctionCallExpression) unnestOp.getExpressionRef().getValue();
+        if (functionCall.getFunctionIdentifier() != BuiltinFunctions.FEED_COLLECT) {
+            return false;
+        }
+        return operation == DatasetUtil.OP_UPSERT;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
index 7ff423c..1d1db37 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Function.java
@@ -26,6 +26,7 @@ import org.apache.asterix.metadata.api.IMetadataEntity;
 public class Function implements IMetadataEntity<Function> {
     private static final long serialVersionUID = 1L;
     public static final String LANGUAGE_AQL = "AQL";
+    public static final String LANGUAGE_SQLPP = "SQLPP";
     public static final String LANGUAGE_JAVA = "JAVA";
 
     public static final String RETURNTYPE_VOID = "VOID";


[2/3] asterixdb git commit: [ASTERIXDB-1983] Feed pipeline refactoring for SQL++

Posted by xi...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.4.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.4.sleep.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.4.sleep.aql
deleted file mode 100644
index 99ad0d7..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.4.sleep.aql
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-10000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.5.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.5.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.5.update.aql
deleted file mode 100644
index 30d3274..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.5.update.aql
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-use dataverse experiments;
-stop feed UserFeed;
-disconnect feed UserFeed from dataset TwitterUsers;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.6.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.6.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.6.query.aql
deleted file mode 100644
index 05d0cee..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.6.query.aql
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-
-use dataverse experiments;
-
-for $x in dataset TwitterUsers
-order by $x.screen-name
-return $x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.7.server.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.7.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.7.server.aql
deleted file mode 100644
index d331474..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.7.server.aql
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-
-stop 10001
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.8.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.8.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.8.ddl.aql
deleted file mode 100644
index d556f2c..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.8.ddl.aql
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-use dataverse experiments;
-drop dataverse experiments;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/single-line-definition/single-line-definition.2.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/single-line-definition/single-line-definition.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/single-line-definition/single-line-definition.2.query.aql
index 087788a..5377919 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/single-line-definition/single-line-definition.2.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/single-line-definition/single-line-definition.2.query.aql
@@ -23,6 +23,12 @@
  * Date         : Jul 10th 2016
  */
 
-for $x in dataset Metadata.Function
-where $x.DataverseName = "test"
-return $x;
+for $l in dataset('Metadata.Function')
+where $l.DataverseName='test'
+return {
+"DataverseName": $l.DataverseName,
+"Name": $l.Name,
+"Arity": $l.Arity,
+"ReturnType": $l.ReturnType,
+"Definition": $l.Definition
+};

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf28/udf28.2.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf28/udf28.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf28/udf28.2.query.aql
index 30050cf..3f4042e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf28/udf28.2.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf28/udf28.2.query.aql
@@ -22,7 +22,12 @@
  * Date         : Apr 5th 2013
  */
 
-use dataverse test;
-for $x in dataset Metadata.Function
-where $x.DataverseName='test'
-return $x
+for $l in dataset('Metadata.Function')
+where $l.DataverseName='test'
+return {
+"DataverseName": $l.DataverseName,
+"Name": $l.Name,
+"Arity": $l.Arity,
+"ReturnType": $l.ReturnType,
+"Definition": $l.Definition
+};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv15/cross-dv15.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv15/cross-dv15.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv15/cross-dv15.3.query.sqlpp
index e002a16..264e918 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv15/cross-dv15.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/cross-dataverse/cross-dv15/cross-dv15.3.query.sqlpp
@@ -27,8 +27,7 @@ select element {
 'DataverseName': l.DataverseName,
 'Name': l.Name,
 'Arity': l.Arity,
-'ReturnType': l.ReturnType,
-'Language': l.Language
+'ReturnType': l.ReturnType
 }
 from  `Metadata.Function` as l
 where (l.DataverseName = 'testdv1')

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.1.ddl.sqlpp
new file mode 100644
index 0000000..4de5415
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.1.ddl.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed and apply two functions in the
+ * workflow. The output of the first function can be used in
+ * the second function. The function parameter can have any
+ * name.
+ * Expected Res : Success
+ * Date         : 4th Oct 2017
+ */
+drop dataverse experiments if exists;
+create dataverse experiments;
+use experiments;
+
+create type TwitterUser if not exists as open{
+    `screen-name`: string,
+    friends_count: int32,
+    name: string,
+    followers_count: int32
+};
+
+create dataset TwitterUsers(TwitterUser) primary key `screen-name`;
+
+create function test_func0(xyz) {
+    object_merge((case (xyz.followers_count > 25000) when true then {"popularity":"Good!"} else {"popularity":"Bad!"} end), xyz)
+};
+
+create function test_func1(anyname) {
+    object_merge((case (anyname.popularity = "Good!") when true then {"true_popularity":"Indeed Good!"} else {"true_popularity":"Indeed Bad!"} end), anyname)
+};
+
+create feed UserFeed using socket_adapter
+(
+    ("sockets"="127.0.0.1:10001"),
+    ("address-type"="IP"),
+    ("type-name"="TwitterUser"),
+    ("format"="adm"),
+    ("upsert-feed"="true")
+);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.2.update.sqlpp
new file mode 100644
index 0000000..9606396
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use experiments;
+
+connect feed UserFeed to dataset TwitterUsers apply function test_func0,test_func1;
+
+start feed UserFeed;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.3.server.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.3.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.3.server.sqlpp
new file mode 100644
index 0000000..eacf623
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.3.server.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed and apply two functions in the
+ * workflow. The output of the first function can be used in
+ * the second function. The function parameter can have any
+ * name.
+ * Expected Res : Success
+ * Date         : 29th Mar 2017
+ */
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/tinysocial/twu.adm 500 50 1000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.4.sleep.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.4.sleep.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.4.sleep.sqlpp
new file mode 100644
index 0000000..e50b429
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.4.sleep.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+2000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.5.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.5.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.5.update.sqlpp
new file mode 100644
index 0000000..136d142
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.5.update.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use experiments;
+stop feed UserFeed;
+disconnect feed UserFeed from dataset TwitterUsers;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.6.query.sqlpp
new file mode 100644
index 0000000..4b1dba7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.6.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use experiments;
+
+select value t.true_popularity from TwitterUsers t
+order by t.`screen-name`;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.7.server.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.7.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.7.server.sqlpp
new file mode 100644
index 0000000..c3ba795
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.7.server.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+stop 10001
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.8.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.8.ddl.sqlpp
new file mode 100644
index 0000000..ec08f08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/connect-feed-with-function/connect-feed-with-function.8.ddl.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+drop dataverse experiments;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.1.ddl.sqlpp
new file mode 100644
index 0000000..9b0f2a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.1.ddl.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Apply user defined function to feed.
+ * Expected Res : Success
+ * Date         : 4th Oct 2017
+ */
+
+drop dataverse externallibtest if exists;
+create dataverse externallibtest;
+use externallibtest;
+
+create type TweetInputType as open {
+  id: string,
+  username : string,
+  location : string,
+  text : string,
+  timestamp : string
+};
+
+create type TweetOutputType as open {
+  id: string,
+  username : string,
+  location : string,
+  text : string,
+  timestamp : string,
+  topics : {{string}}
+};
+
+create feed TweetFeed
+using localfs
+(("type-name"="TweetInputType"),
+("path"="asterix_nc1://data/twitter/obamatweets.adm"),
+("format"="adm"));
+
+create dataset TweetsFeedIngest(TweetOutputType)
+primary key id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.2.lib.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.2.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.2.lib.sqlpp
new file mode 100644
index 0000000..d1e0e87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.2.lib.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+install externallibtest testlib target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.3.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.3.update.sqlpp
new file mode 100644
index 0000000..883cd7a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.3.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Apply user defined function to feed.
+ * Expected Res : Success
+ * Date         : 4th Oct 2017
+ */
+use externallibtest;
+
+connect feed TweetFeed to dataset TweetsFeedIngest apply function `testlib#parseTweet`;
+
+start feed TweetFeed;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.5.pollquery.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.5.pollquery.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.5.pollquery.sqlpp
new file mode 100644
index 0000000..607e5bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.5.pollquery.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Apply user defined function to feed.
+ * Expected Res : Success
+ * Date         : 4th Oct 2017
+ */
+// polltimeoutsecs=5
+use externallibtest;
+
+select value t from TweetsFeedIngest t
+ORDER BY t.id;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.6.lib.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.6.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.6.lib.sqlpp
new file mode 100644
index 0000000..86af80f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.6.lib.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+uninstall externallibtest testlib
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.7.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.7.ddl.sqlpp
new file mode 100644
index 0000000..2a7acef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.7.ddl.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+drop dataverse externallibtest if exists;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.1.ddl.sqlpp
new file mode 100644
index 0000000..be7c383
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.1.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+drop dataverse experiments if exists;
+create dataverse experiments;
+use experiments;
+
+create type TwitterUser if not exists as open{
+    `screen-name`: string,
+    friends_count: int32,
+    name: string,
+    followers_count: int32
+};
+
+create dataset TwitterUsers(TwitterUser) primary key `screen-name`;
+
+create feed UserFeed using socket_adapter
+(
+    ("sockets"="127.0.0.1:10001"),
+    ("address-type"="IP"),
+    ("type-name"="TwitterUser"),
+    ("format"="adm"),
+    ("insert-feed"="true")
+);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.2.update.sqlpp
new file mode 100644
index 0000000..3bca1a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.2.update.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+use experiments;
+set `wait-for-completion-feed` "false";
+
+connect feed UserFeed to dataset TwitterUsers;
+start feed UserFeed;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.3.server.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.3.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.3.server.sqlpp
new file mode 100644
index 0000000..3da77f7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.3.server.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+start client 10001 file-client localhost ../asterix-app/data/tinysocial/twu_update.adm 500 50 1000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.4.sleep.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.4.sleep.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.4.sleep.sqlpp
new file mode 100644
index 0000000..99ad0d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.4.sleep.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+10000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.5.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.5.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.5.update.sqlpp
new file mode 100644
index 0000000..c19b2f7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.5.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+
+use experiments;
+stop feed UserFeed;
+disconnect feed UserFeed from dataset TwitterUsers;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.6.query.sqlpp
new file mode 100644
index 0000000..77ee294
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.6.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+
+use experiments;
+
+select value x
+from TwitterUsers x
+order by x.`screen-name`;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.7.server.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.7.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.7.server.sqlpp
new file mode 100644
index 0000000..d331474
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.7.server.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+
+stop 10001
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.8.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.8.ddl.sqlpp
new file mode 100644
index 0000000..2d29f2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/insert-feed/insert-feed.8.ddl.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+use experiments;
+drop dataverse experiments;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.1.ddl.sqlpp
deleted file mode 100644
index 4573f89..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.1.ddl.sqlpp
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-drop dataverse experiments if exists;
-create dataverse experiments;
-use experiments;
-
-create type TwitterUser if not exists as open{
-    `screen-name`: string,
-    friends_count: int32,
-    name: string,
-    followers_count: int32
-};
-
-create dataset TwitterUsers(TwitterUser) primary key `screen-name`;
-
-create feed UserFeed using socket_adapter
-(
-    ("sockets"="127.0.0.1:10001"),
-    ("address-type"="IP"),
-    ("type-name"="TwitterUser"),
-    ("format"="adm"),
-    ("upsert-feed"="true")
-);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.2.update.sqlpp
deleted file mode 100644
index 3bca1a2..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.2.update.sqlpp
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-use experiments;
-set `wait-for-completion-feed` "false";
-
-connect feed UserFeed to dataset TwitterUsers;
-start feed UserFeed;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.3.server.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.3.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.3.server.sqlpp
deleted file mode 100644
index 3da77f7..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.3.server.sqlpp
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-start client 10001 file-client localhost ../asterix-app/data/tinysocial/twu_update.adm 500 50 1000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.4.sleep.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.4.sleep.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.4.sleep.sqlpp
deleted file mode 100644
index 99ad0d7..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.4.sleep.sqlpp
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-10000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.5.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.5.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.5.update.sqlpp
deleted file mode 100644
index c19b2f7..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.5.update.sqlpp
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-
-use experiments;
-stop feed UserFeed;
-disconnect feed UserFeed from dataset TwitterUsers;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.6.query.sqlpp
deleted file mode 100644
index 77ee294..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.6.query.sqlpp
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-
-use experiments;
-
-select value x
-from TwitterUsers x
-order by x.`screen-name`;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.7.server.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.7.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.7.server.sqlpp
deleted file mode 100644
index d331474..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.7.server.sqlpp
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-
-stop 10001
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.8.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.8.ddl.sqlpp
deleted file mode 100644
index 2d29f2f..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.8.ddl.sqlpp
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-use experiments;
-drop dataverse experiments;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/single-line-definition/single-line-definition.2.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/single-line-definition/single-line-definition.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/single-line-definition/single-line-definition.2.query.sqlpp
index 9763645..6e47c62 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/single-line-definition/single-line-definition.2.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/single-line-definition/single-line-definition.2.query.sqlpp
@@ -23,8 +23,13 @@
  * Date         : Jul 10th 2016
  */
 
-use Metadata;
-
-select value `Function`
-from `Function`
-where DataverseName = 'test';
\ No newline at end of file
+select element {
+'DataverseName': l.DataverseName,
+'Name': l.Name,
+'Arity': l.Arity,
+'ReturnType': l.ReturnType,
+'Definition': l.`Definition`
+}
+from  `Metadata.Function` as l
+where (l.DataverseName = 'test')
+;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf28/udf28.2.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf28/udf28.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf28/udf28.2.query.sqlpp
index cea1b28..1083570 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf28/udf28.2.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf28/udf28.2.query.sqlpp
@@ -22,10 +22,13 @@
  * Date         : Apr 5th 2013
  */
 
-use test;
-
-
-select element x
-from  `Metadata.Function` as x
-where (x.DataverseName = 'test')
-;
+select element {
+'DataverseName': l.DataverseName,
+'Name': l.Name,
+'Arity': l.Arity,
+'ReturnType': l.ReturnType,
+'Definition': l.`Definition`
+}
+from  `Metadata.Function` as l
+where (l.DataverseName = 'test')
+;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv15/cross-dv15.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv15/cross-dv15.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv15/cross-dv15.1.adm
index f6b9984..3b6f8538 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv15/cross-dv15.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv15/cross-dv15.1.adm
@@ -1,3 +1,3 @@
-{ "DataverseName": "testdv1", "Name": "fun01", "Arity": "0", "ReturnType": "VOID", "Language": "AQL" }
-{ "DataverseName": "testdv1", "Name": "fun02", "Arity": "1", "ReturnType": "VOID", "Language": "AQL" }
-{ "DataverseName": "testdv1", "Name": "fun03", "Arity": "2", "ReturnType": "VOID", "Language": "AQL" }
+{ "DataverseName": "testdv1", "Name": "fun01", "Arity": "0", "ReturnType": "VOID" }
+{ "DataverseName": "testdv1", "Name": "fun02", "Arity": "1", "ReturnType": "VOID" }
+{ "DataverseName": "testdv1", "Name": "fun03", "Arity": "2", "ReturnType": "VOID" }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.1.adm
deleted file mode 100644
index c5a10be..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.1.adm
+++ /dev/null
@@ -1,4 +0,0 @@
-"Indeed Good!"
-"Indeed Bad!"
-"Indeed Good!"
-"Indeed Bad!"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/connect-feed-with-function/connect-feed-with-function.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/connect-feed-with-function/connect-feed-with-function.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/connect-feed-with-function/connect-feed-with-function.1.adm
new file mode 100644
index 0000000..c5a10be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/connect-feed-with-function/connect-feed-with-function.1.adm
@@ -0,0 +1,4 @@
+"Indeed Good!"
+"Indeed Bad!"
+"Indeed Good!"
+"Indeed Bad!"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
index fbd87b6..e8fba07 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_03/feeds_03.1.adm
@@ -1 +1 @@
-{ "DataverseName": "feeds", "FeedName": "TweetFeed", "DatasetName": "Tweets", "ReturnType": "TweetType: closed {\n  id: string,\n  username: string,\n  location: string,\n  text: string,\n  timestamp: string\n}\n", "AppliedFunctions": {{ "feeds.feed_processor" }}, "PolicyName": "Basic" }
+{ "DataverseName": "feeds", "FeedName": "TweetFeed", "DatasetName": "Tweets", "ReturnType": "TweetType", "AppliedFunctions": {{ "feeds.feed_processor" }}, "PolicyName": "Basic" }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/insert-feed/insert-feed.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/insert-feed/insert-feed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/insert-feed/insert-feed.1.adm
new file mode 100644
index 0000000..ff01ac6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/insert-feed/insert-feed.1.adm
@@ -0,0 +1,6 @@
+{ "screen-name": "ChangEwing_573", "friends_count": 182, "name": "Chang Ewing", "followers_count": 32136, "lang": "en", "statuses_count": 394 }
+{ "screen-name": "ColineGeyer@63", "friends_count": 121, "name": "Coline Geyer", "followers_count": 17159, "lang": "en", "statuses_count": 362 }
+{ "screen-name": "Farrel@64", "friends_count": 42, "name": "Will Farrel", "followers_count": 1422, "lang": "en", "statuses_count": 13 }
+{ "screen-name": "NathanGiesen@211", "friends_count": 18, "name": "Nathan Giesen", "followers_count": 49416, "lang": "en", "statuses_count": 473 }
+{ "screen-name": "NilaMilliron_tw", "friends_count": 445, "name": "Nila Milliron", "followers_count": 22649, "lang": "en", "statuses_count": 164 }
+{ "screen-name": "Zeed", "friends_count": 32, "name": "Steven Zeed", "followers_count": 331, "lang": "en", "statuses_count": 13 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/upsert-feed/upsert-feed.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/upsert-feed/upsert-feed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/upsert-feed/upsert-feed.1.adm
deleted file mode 100644
index 2d0a865..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/upsert-feed/upsert-feed.1.adm
+++ /dev/null
@@ -1,6 +0,0 @@
-{ "screen-name": "ChangEwing_573", "friends_count": 182, "name": "Chang Ewing", "followers_count": 32136, "lang": "en", "statuses_count": 394 }
-{ "screen-name": "ColineGeyer@63", "friends_count": 121, "name": "Coline Geyer", "followers_count": 17159, "lang": "en", "statuses_count": 362 }
-{ "screen-name": "Farrel@64", "friends_count": 42, "name": "Will Farrel", "followers_count": 1422, "lang": "en", "statuses_count": 13 }
-{ "screen-name": "NathanGiesen@211", "friends_count": 18, "name": "Nathan Giesen", "followers_count": 4, "lang": "en", "statuses_count": 473 }
-{ "screen-name": "NilaMilliron_tw", "friends_count": 455, "name": "Nila Milliron", "followers_count": 1, "lang": "en", "statuses_count": 164 }
-{ "screen-name": "Zeed", "friends_count": 32, "name": "Steven Zeed", "followers_count": 331, "lang": "en", "statuses_count": 13 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/single-line-definition/single-line-definition.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/single-line-definition/single-line-definition.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/single-line-definition/single-line-definition.1.adm
index 5ba00ae..7b4987c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/single-line-definition/single-line-definition.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/single-line-definition/single-line-definition.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "Name": "printName", "Arity": "0", "Params": [  ], "ReturnType": "VOID", "Definition": "'AsterixDB Shared nothing parallel BDMS'", "Language": "AQL", "Kind": "SCALAR" }
+{ "DataverseName": "test", "Name": "printName", "Arity": "0", "ReturnType": "VOID", "Definition": "'AsterixDB Shared nothing parallel BDMS'" }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf28/udf28.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf28/udf28.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf28/udf28.1.adm
index b106dbe..46af45a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf28/udf28.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf28/udf28.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "Name": "f1", "Arity": "0", "Params": [  ], "ReturnType": "VOID", "Definition": "100", "Language": "AQL", "Kind": "SCALAR" }
+{ "DataverseName": "test", "Name": "f1", "Arity": "0", "ReturnType": "VOID", "Definition": "100" }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv15/cross-dv15.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv15/cross-dv15.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv15/cross-dv15.3.ast
index e4a2db3..4fd48a7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv15/cross-dv15.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv15/cross-dv15.3.ast
@@ -33,14 +33,6 @@ RecordConstructor [
       Field=ReturnType
     ]
   )
-  (
-    LiteralExpr [STRING] [Language]
-    :
-    FieldAccessor [
-      Variable [ Name=$l ]
-      Field=Language
-    ]
-  )
 ]
 ]
 FROM [  FunctionCall Metadata.dataset@1[

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/user-defined-functions/udf28/udf28.2.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/user-defined-functions/udf28/udf28.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/user-defined-functions/udf28/udf28.2.ast
index 95a8e86..17e46ec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/user-defined-functions/udf28/udf28.2.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/user-defined-functions/udf28/udf28.2.ast
@@ -1,17 +1,57 @@
-DataverseUse test
 Query:
 SELECT ELEMENT [
-Variable [ Name=$x ]
+RecordConstructor [
+  (
+    LiteralExpr [STRING] [DataverseName]
+    :
+    FieldAccessor [
+      Variable [ Name=$l ]
+      Field=DataverseName
+    ]
+  )
+  (
+    LiteralExpr [STRING] [Name]
+    :
+    FieldAccessor [
+      Variable [ Name=$l ]
+      Field=Name
+    ]
+  )
+  (
+    LiteralExpr [STRING] [Arity]
+    :
+    FieldAccessor [
+      Variable [ Name=$l ]
+      Field=Arity
+    ]
+  )
+  (
+    LiteralExpr [STRING] [ReturnType]
+    :
+    FieldAccessor [
+      Variable [ Name=$l ]
+      Field=ReturnType
+    ]
+  )
+  (
+    LiteralExpr [STRING] [Definition]
+    :
+    FieldAccessor [
+      Variable [ Name=$l ]
+      Field=Definition
+    ]
+  )
+]
 ]
 FROM [  FunctionCall Metadata.dataset@1[
     LiteralExpr [STRING] [Metadata.Function]
   ]
-  AS Variable [ Name=$x ]
+  AS Variable [ Name=$l ]
 ]
 Where
   OperatorExpr [
     FieldAccessor [
-      Variable [ Name=$x ]
+      Variable [ Name=$l ]
       Field=DataverseName
     ]
     =

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index abbcaaa..47560f6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -252,8 +252,9 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">
-      <compilation-unit name="connect-feed-with-aql-function">
-        <output-dir compare="Text">connect-feed-with-aql-function</output-dir>
+      <compilation-unit name="connect-feed-with-function">
+        <output-dir compare="Text">connect-feed-with-function</output-dir>
+        <expected-error>Incompatible function language</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="feeds">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
index 83cdd82..8f4455f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
@@ -43,5 +43,10 @@
         <output-dir compare="Text">upperCase</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="feed-with-external-function">
+        <output-dir compare="Text">feed-with-external-function</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
 </test-suite>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index c87c44b..0d7ef36 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -8248,6 +8248,11 @@
         <output-dir compare="Text">upsert-feed</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="connect-feed-with-function">
+        <output-dir compare="Text">connect-feed-with-function</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="hdfs">
     <test-case FilePath="hdfs">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index f960ce5..279624d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -129,6 +129,7 @@ public class ErrorCode {
     public static final int INDEX_ILLEGAL_REPETITIVE_FIELD = 1052;
     public static final int CANNOT_CREATE_SEC_PRIMARY_IDX_ON_EXT_DATASET = 1053;
     public static final int COMPILATION_FAILED_DUE_TO_REPLICATE_OP = 1054;
+    public static final int COMPILATION_INCOMPATIBLE_FUNCTION_LANGUAGE = 1055;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 7362181..6ce78f0 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -115,6 +115,7 @@
 1052 = Cannot create index with the same field \"%1$s\" specified more than once.
 1053 = Cannot create primary index on external dataset.
 1054 = Compilation failed due to some problem in the query plan.
+1055 = Incompatible function language. Expect %1$s, but %2$s found.
 
 # Feed Errors
 3001 = Illegal state.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index 3b6e7ff..a6c46c3 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -87,7 +87,7 @@ public class ExternalDataConstants {
     // a boolean indicating whether the feed is a change feed
     public static final String KEY_IS_CHANGE_FEED = "change-feed";
     // a boolean indicating whether the feed use upsert
-    public static final String KEY_IS_UPSERT_FEED = "upsert-feed";
+    public static final String KEY_IS_INSERT_FEED = "insert-feed";
     // an integer representing the number of keys in a change feed
     public static final String KEY_KEY_SIZE = "key-size";
     // a boolean indicating whether the feed produces records with metadata


[3/3] asterixdb git commit: [ASTERIXDB-1983] Feed pipeline refactoring for SQL++

Posted by xi...@apache.org.
[ASTERIXDB-1983] Feed pipeline refactoring for SQL++

- user model changes: no
- storage format changes: no
- interface changes: no

Current implementation of feed uses handcraft AQL queries for creating
feed pipeline. This causes a lot of issues and does not support SQL++
very well. Also, there is an overhead for parsing the query everytime.
In this patch, it's replaced with compiled statement in SQL++ which
provides support for attaching UDF to feed as well.

Details:
1. Remove SubscribeFeedStatement.
2. Remove SubscribeFeed related query compilation code, and reuse the
upsert dataflow.
3. Added SQL++ User Defined Function support for feed, including adding
multiple functions to one feed.
4. Related test cases added.
5. Change the default behavior of feed to be upsert instead of upsert.
'insert-feed' option is provided for experiment uses.
6. This patch also fixes several feed related bugs: [ASTERIXDB-2085]
[ASTERIXDB-2124].

Change-Id: I0ae5a837613780a4d2c90c98139fdc6d5e040cc9
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2059
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <ba...@gmail.com>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/7e76a079
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/7e76a079
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/7e76a079

Branch: refs/heads/master
Commit: 7e76a0797388d5c8f92a96c7fa6b4cf30c0886f0
Parents: 799046d
Author: Xikui Wang <xk...@gmail.com>
Authored: Fri Oct 20 13:12:30 2017 -0700
Committer: Xikui Wang <xk...@gmail.com>
Committed: Sat Oct 21 09:10:23 2017 -0700

----------------------------------------------------------------------
 .../asterix/translator/CompiledStatements.java  |  82 --------
 .../LangExpressionToPlanTranslator.java         | 119 +-----------
 .../apache/asterix/api/common/APIFramework.java |   4 +-
 .../asterix/app/active/FeedEventsListener.java  |  16 +-
 .../asterix/app/translator/QueryTranslator.java |   7 +-
 .../apache/asterix/utils/FeedOperations.java    | 163 ++++++++++++----
 .../cross-dv15/cross-dv15.3.query.aql           |   3 +-
 .../connect-feed-with-aql-function.1.ddl.aql    |  57 ------
 .../connect-feed-with-aql-function.2.update.aql |  32 ----
 .../connect-feed-with-aql-function.3.server.aql |  27 ---
 .../connect-feed-with-aql-function.4.sleep.aql  |  27 ---
 .../connect-feed-with-aql-function.5.update.aql |  29 ---
 .../connect-feed-with-aql-function.6.query.aql  |  31 ---
 .../connect-feed-with-aql-function.7.server.aql |  28 ---
 .../connect-feed-with-aql-function.8.ddl.aql    |  28 ---
 .../connect-feed-with-function.1.ddl.aql        |  57 ++++++
 .../connect-feed-with-function.2.update.aql     |  32 ++++
 .../connect-feed-with-function.3.ddl.aql        |  27 +++
 .../feeds/insert-feed/insert-feed.1.ddl.aql     |  45 +++++
 .../feeds/insert-feed/insert-feed.2.update.aql  |  24 +++
 .../feeds/insert-feed/insert-feed.3.server.aql  |  19 ++
 .../feeds/insert-feed/insert-feed.4.sleep.aql   |  19 ++
 .../feeds/insert-feed/insert-feed.5.update.aql  |  21 +++
 .../feeds/insert-feed/insert-feed.6.query.aql   |  24 +++
 .../feeds/insert-feed/insert-feed.7.server.aql  |  20 ++
 .../feeds/insert-feed/insert-feed.8.ddl.aql     |  20 ++
 .../feeds/upsert-feed/upsert-feed.1.ddl.aql     |  46 -----
 .../feeds/upsert-feed/upsert-feed.2.update.aql  |  31 ---
 .../feeds/upsert-feed/upsert-feed.3.server.aql  |  26 ---
 .../feeds/upsert-feed/upsert-feed.4.sleep.aql   |  26 ---
 .../feeds/upsert-feed/upsert-feed.5.update.aql  |  28 ---
 .../feeds/upsert-feed/upsert-feed.6.query.aql   |  31 ---
 .../feeds/upsert-feed/upsert-feed.7.server.aql  |  27 ---
 .../feeds/upsert-feed/upsert-feed.8.ddl.aql     |  27 ---
 .../single-line-definition.2.query.aql          |  12 +-
 .../udf28/udf28.2.query.aql                     |  13 +-
 .../cross-dv15/cross-dv15.3.query.sqlpp         |   3 +-
 .../connect-feed-with-function.1.ddl.sqlpp      |  55 ++++++
 .../connect-feed-with-function.2.update.sqlpp   |  23 +++
 .../connect-feed-with-function.3.server.sqlpp   |  27 +++
 .../connect-feed-with-function.4.sleep.sqlpp    |  20 ++
 .../connect-feed-with-function.5.update.sqlpp   |  21 +++
 .../connect-feed-with-function.6.query.sqlpp    |  22 +++
 .../connect-feed-with-function.7.server.sqlpp   |  19 ++
 .../connect-feed-with-function.8.ddl.sqlpp      |  19 ++
 .../feed-with-external-function.1.ddl.sqlpp     |  53 ++++++
 .../feed-with-external-function.2.lib.sqlpp     |  19 ++
 .../feed-with-external-function.3.update.sqlpp  |  28 +++
 ...eed-with-external-function.5.pollquery.sqlpp |  29 +++
 .../feed-with-external-function.6.lib.sqlpp     |  19 ++
 .../feed-with-external-function.7.ddl.sqlpp     |  19 ++
 .../feeds/insert-feed/insert-feed.1.ddl.sqlpp   |  46 +++++
 .../insert-feed/insert-feed.2.update.sqlpp      |  30 +++
 .../insert-feed/insert-feed.3.server.sqlpp      |  26 +++
 .../feeds/insert-feed/insert-feed.4.sleep.sqlpp |  26 +++
 .../insert-feed/insert-feed.5.update.sqlpp      |  29 +++
 .../feeds/insert-feed/insert-feed.6.query.sqlpp |  31 +++
 .../insert-feed/insert-feed.7.server.sqlpp      |  27 +++
 .../feeds/insert-feed/insert-feed.8.ddl.sqlpp   |  27 +++
 .../feeds/upsert-feed/upsert-feed.1.ddl.sqlpp   |  46 -----
 .../upsert-feed/upsert-feed.2.update.sqlpp      |  30 ---
 .../upsert-feed/upsert-feed.3.server.sqlpp      |  26 ---
 .../feeds/upsert-feed/upsert-feed.4.sleep.sqlpp |  26 ---
 .../upsert-feed/upsert-feed.5.update.sqlpp      |  29 ---
 .../feeds/upsert-feed/upsert-feed.6.query.sqlpp |  31 ---
 .../upsert-feed/upsert-feed.7.server.sqlpp      |  27 ---
 .../feeds/upsert-feed/upsert-feed.8.ddl.sqlpp   |  27 ---
 .../single-line-definition.2.query.sqlpp        |  15 +-
 .../udf28/udf28.2.query.sqlpp                   |  17 +-
 .../cross-dataverse/cross-dv15/cross-dv15.1.adm |   6 +-
 .../connect-feed-with-aql-function.1.adm        |   4 -
 .../connect-feed-with-function.1.adm            |   4 +
 .../results/feeds/feeds_03/feeds_03.1.adm       |   2 +-
 .../results/feeds/insert-feed/insert-feed.1.adm |   6 +
 .../results/feeds/upsert-feed/upsert-feed.1.adm |   6 -
 .../single-line-definition.1.adm                |   2 +-
 .../user-defined-functions/udf28/udf28.1.adm    |   2 +-
 .../cross-dataverse/cross-dv15/cross-dv15.3.ast |   8 -
 .../user-defined-functions/udf28/udf28.2.ast    |  48 ++++-
 .../src/test/resources/runtimets/testsuite.xml  |   5 +-
 .../resources/runtimets/testsuite_it_sqlpp.xml  |   5 +
 .../resources/runtimets/testsuite_sqlpp.xml     |   5 +
 .../asterix/common/exceptions/ErrorCode.java    |   1 +
 .../main/resources/asx_errormsg/en.properties   |   1 +
 .../external/util/ExternalDataConstants.java    |   2 +-
 .../external/util/ExternalDataUtils.java        |   4 +-
 .../asterix/external/util/FeedConstants.java    |   7 +-
 asterixdb/asterix-lang-aql/pom.xml              |  10 -
 .../asterix/lang/aql/parser/FunctionParser.java |   5 +
 .../aql/statement/SubscribeFeedStatement.java   | 189 -------------------
 .../asterix/lang/common/util/FunctionUtil.java  |   3 +-
 .../lang/sqlpp/parser/FunctionParser.java       |   5 +
 .../asterix/metadata/entities/Dataset.java      |  32 +++-
 .../asterix/metadata/entities/Function.java     |   1 +
 94 files changed, 1229 insertions(+), 1254 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
index 403c26b..068aa29 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
@@ -320,56 +320,6 @@ public class CompiledStatements {
         }
     }
 
-    public static class CompiledConnectFeedStatement implements ICompiledDmlStatement {
-        private final String dataverseName;
-        private final String feedName;
-        private final String datasetName;
-        private final String policyName;
-        private final Query query;
-        private final int varCounter;
-
-        public CompiledConnectFeedStatement(String dataverseName, String feedName, String datasetName,
-                String policyName, Query query, int varCounter) {
-            this.dataverseName = dataverseName;
-            this.feedName = feedName;
-            this.datasetName = datasetName;
-            this.policyName = policyName;
-            this.query = query;
-            this.varCounter = varCounter;
-        }
-
-        @Override
-        public String getDataverseName() {
-            return dataverseName;
-        }
-
-        public String getFeedName() {
-            return feedName;
-        }
-
-        @Override
-        public String getDatasetName() {
-            return datasetName;
-        }
-
-        public int getVarCounter() {
-            return varCounter;
-        }
-
-        public Query getQuery() {
-            return query;
-        }
-
-        @Override
-        public byte getKind() {
-            return Statement.Kind.CONNECT_FEED;
-        }
-
-        public String getPolicyName() {
-            return policyName;
-        }
-    }
-
     public static class CompiledSubscribeFeedStatement implements ICompiledDmlStatement {
 
         private FeedConnectionRequest request;
@@ -404,38 +354,6 @@ public class CompiledStatements {
         }
     }
 
-    public static class CompiledDisconnectFeedStatement implements ICompiledDmlStatement {
-        private final String dataverseName;
-        private final String datasetName;
-        private final String feedName;
-
-        public CompiledDisconnectFeedStatement(String dataverseName, String feedName, String datasetName) {
-            this.dataverseName = dataverseName;
-            this.feedName = feedName;
-            this.datasetName = datasetName;
-        }
-
-        @Override
-        public String getDataverseName() {
-            return dataverseName;
-        }
-
-        @Override
-        public String getDatasetName() {
-            return datasetName;
-        }
-
-        public String getFeedName() {
-            return feedName;
-        }
-
-        @Override
-        public byte getKind() {
-            return Statement.Kind.DISCONNECT_FEED;
-        }
-
-    }
-
     public static class CompiledDeleteStatement implements ICompiledDmlStatement {
         private final String dataverseName;
         private final String datasetName;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index a1c5cf4..66a1073 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -409,15 +409,6 @@ class LangExpressionToPlanTranslator
                     leafOperator = translateDelete(targetDatasource, varRef, varRefsForLoading,
                             additionalFilteringExpressions, assign);
                     break;
-                case Statement.Kind.CONNECT_FEED:
-                    leafOperator = translateConnectFeed(targetDatasource, varRef, varRefsForLoading,
-                            additionalFilteringExpressions, assign);
-                    break;
-                case Statement.Kind.SUBSCRIBE_FEED:
-                    leafOperator = translateSubscribeFeed((CompiledSubscribeFeedStatement) stmt, targetDatasource,
-                            unnestVar, topOp, exprs, resVar, varRefsForLoading, varRef, assign,
-                            additionalFilteringField, additionalFilteringAssign, additionalFilteringExpressions);
-                    break;
                 default:
                     throw new AlgebricksException("Unsupported statement kind " + stmt.getKind());
             }
@@ -429,18 +420,6 @@ class LangExpressionToPlanTranslator
         return plan;
     }
 
-    private ILogicalOperator translateConnectFeed(DatasetDataSource targetDatasource,
-            Mutable<ILogicalExpression> varRef, List<Mutable<ILogicalExpression>> varRefsForLoading,
-            List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign) {
-        InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
-                varRefsForLoading, InsertDeleteUpsertOperator.Kind.INSERT, false);
-        insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
-        insertOp.getInputs().add(new MutableObject<>(assign));
-        ILogicalOperator leafOperator = new DelegateOperator(new CommitOperator(true));
-        leafOperator.getInputs().add(new MutableObject<>(insertOp));
-        return leafOperator;
-    }
-
     private ILogicalOperator translateDelete(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
             List<Mutable<ILogicalExpression>> varRefsForLoading,
             List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign)
@@ -458,100 +437,6 @@ class LangExpressionToPlanTranslator
         return leafOperator;
     }
 
-    private ILogicalOperator translateSubscribeFeed(CompiledSubscribeFeedStatement sfs,
-            DatasetDataSource targetDatasource, LogicalVariable unnestVar, ILogicalOperator topOp,
-            ArrayList<Mutable<ILogicalExpression>> exprs, LogicalVariable resVar,
-            List<Mutable<ILogicalExpression>> varRefsForLoading, Mutable<ILogicalExpression> varRef,
-            ILogicalOperator assign, List<String> additionalFilteringField, AssignOperator additionalFilteringAssign,
-            List<Mutable<ILogicalExpression>> additionalFilteringExpressions) throws AlgebricksException {
-        // if the feed is a change feed (i.e, performs different operations), we need to project op variable
-        InsertDeleteUpsertOperator feedModificationOp;
-        AssignOperator metaAndKeysAssign;
-        List<LogicalVariable> metaAndKeysVars = null;
-        List<Mutable<ILogicalExpression>> metaAndKeysExprs = null;
-        List<Mutable<ILogicalExpression>> metaExpSingletonList = null;
-        Feed feed = metadataProvider.findFeed(sfs.getDataverseName(), sfs.getFeedName());
-        boolean isChangeFeed = ExternalDataUtils.isChangeFeed(feed.getAdapterConfiguration());
-        boolean isUpsertFeed = ExternalDataUtils.isUpsertFeed(feed.getAdapterConfiguration());
-
-        ProjectOperator project = (ProjectOperator) topOp;
-        if (targetDatasource.getDataset().hasMetaPart() || isChangeFeed) {
-            metaAndKeysVars = new ArrayList<>();
-            metaAndKeysExprs = new ArrayList<>();
-            if (targetDatasource.getDataset().hasMetaPart()) {
-                // add the meta function
-                IFunctionInfo finfoMeta = FunctionUtil.getFunctionInfo(BuiltinFunctions.META);
-                ScalarFunctionCallExpression metaFunction = new ScalarFunctionCallExpression(finfoMeta,
-                        new MutableObject<>(new VariableReferenceExpression(unnestVar)));
-                // create assign for the meta part
-                LogicalVariable metaVar = context.newVar();
-                metaExpSingletonList = new ArrayList<>(1);
-                metaExpSingletonList.add(new MutableObject<>(new VariableReferenceExpression(metaVar)));
-                metaAndKeysVars.add(metaVar);
-                metaAndKeysExprs.add(new MutableObject<>(metaFunction));
-                project.getVariables().add(metaVar);
-            }
-        }
-        if (isChangeFeed) {
-            varRefsForLoading.clear();
-            for (Mutable<ILogicalExpression> assignExpr : exprs) {
-                if (assignExpr.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                    AbstractFunctionCallExpression funcCall = (AbstractFunctionCallExpression) assignExpr.getValue();
-                    funcCall.substituteVar(resVar, unnestVar);
-                    LogicalVariable pkVar = context.newVar();
-                    metaAndKeysVars.add(pkVar);
-                    metaAndKeysExprs.add(new MutableObject<>(assignExpr.getValue()));
-                    project.getVariables().add(pkVar);
-                    varRefsForLoading.add(new MutableObject<>(new VariableReferenceExpression(pkVar)));
-                }
-            }
-            // A change feed, we don't need the assign to access PKs
-            feedModificationOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading,
-                    metaExpSingletonList, InsertDeleteUpsertOperator.Kind.UPSERT, false);
-            // Create and add a new variable used for representing the original record
-            feedModificationOp.setPrevRecordVar(context.newVar());
-            feedModificationOp.setPrevRecordType(targetDatasource.getItemType());
-            if (targetDatasource.getDataset().hasMetaPart()) {
-                List<LogicalVariable> metaVars = new ArrayList<>();
-                metaVars.add(context.newVar());
-                feedModificationOp.setPrevAdditionalNonFilteringVars(metaVars);
-                List<Object> metaTypes = new ArrayList<>();
-                metaTypes.add(targetDatasource.getMetaItemType());
-                feedModificationOp.setPrevAdditionalNonFilteringTypes(metaTypes);
-            }
-
-            if (additionalFilteringField != null) {
-                feedModificationOp.setPrevFilterVar(context.newVar());
-                feedModificationOp.setPrevFilterType(
-                        ((ARecordType) targetDatasource.getItemType()).getFieldType(additionalFilteringField.get(0)));
-                additionalFilteringAssign.getInputs().clear();
-                additionalFilteringAssign.getInputs().add(assign.getInputs().get(0));
-                feedModificationOp.getInputs().add(new MutableObject<>(additionalFilteringAssign));
-            } else {
-                feedModificationOp.getInputs().add(assign.getInputs().get(0));
-            }
-        } else {
-            final InsertDeleteUpsertOperator.Kind opKind =
-                    isUpsertFeed ? InsertDeleteUpsertOperator.Kind.UPSERT : InsertDeleteUpsertOperator.Kind.INSERT;
-            feedModificationOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading,
-                    metaExpSingletonList, opKind, false);
-            if (isUpsertFeed) {
-                feedModificationOp.setPrevRecordVar(context.newVar());
-                feedModificationOp.setPrevRecordType(targetDatasource.getItemType());
-            }
-            feedModificationOp.getInputs().add(new MutableObject<>(assign));
-        }
-        if (targetDatasource.getDataset().hasMetaPart() || isChangeFeed) {
-            metaAndKeysAssign = new AssignOperator(metaAndKeysVars, metaAndKeysExprs);
-            metaAndKeysAssign.getInputs().add(topOp.getInputs().get(0));
-            topOp.getInputs().set(0, new MutableObject<>(metaAndKeysAssign));
-        }
-        feedModificationOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
-        ILogicalOperator leafOperator = new DelegateOperator(new CommitOperator(true));
-        leafOperator.getInputs().add(new MutableObject<>(feedModificationOp));
-        return leafOperator;
-    }
-
     private ILogicalOperator translateUpsert(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
             List<Mutable<ILogicalExpression>> varRefsForLoading,
             List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign,
@@ -570,7 +455,6 @@ class LangExpressionToPlanTranslator
         if (targetDatasource.getDataset().hasMetaPart()) {
             if (returnExpression != null) {
                 throw new AlgebricksException("Returning not allowed on datasets with Meta records");
-
             }
             AssignOperator metaAndKeysAssign;
             List<LogicalVariable> metaAndKeysVars;
@@ -859,7 +743,8 @@ class LangExpressionToPlanTranslator
             IFunctionInfo finfo = ExternalFunctionCompilerUtil
                     .getExternalFunctionInfo(metadataProvider.getMetadataTxnContext(), function);
             f = new ScalarFunctionCallExpression(finfo, args);
-        } else if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
+        } else if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)
+                || function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_SQLPP)) {
             IFunctionInfo finfo = FunctionUtil.getFunctionInfo(signature);
             f = new ScalarFunctionCallExpression(finfo, args);
         } else {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 583302b..4b78b93 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -50,7 +50,6 @@ import org.apache.asterix.dataflow.data.common.PartialAggregationTypeComputer;
 import org.apache.asterix.external.feed.watch.FeedActivityDetails;
 import org.apache.asterix.formats.base.IDataFormat;
 import org.apache.asterix.jobgen.QueryLogicalExpressionJobGen;
-import org.apache.asterix.lang.aql.statement.SubscribeFeedStatement;
 import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
 import org.apache.asterix.lang.common.base.IQueryRewriter;
 import org.apache.asterix.lang.common.base.IReturningStatement;
@@ -59,6 +58,7 @@ import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.Query;
+import org.apache.asterix.lang.common.statement.StartFeedStatement;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.optimizer.base.FuzzyUtils;
@@ -117,7 +117,7 @@ public class APIFramework {
             ImmutableSet.of(CompilerProperties.COMPILER_JOINMEMORY_KEY, CompilerProperties.COMPILER_GROUPMEMORY_KEY,
                     CompilerProperties.COMPILER_SORTMEMORY_KEY, CompilerProperties.COMPILER_PARALLELISM_KEY,
                     FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, FuzzyUtils.SIM_FUNCTION_PROP_NAME,
-                    FuzzyUtils.SIM_THRESHOLD_PROP_NAME, SubscribeFeedStatement.WAIT_FOR_COMPLETION,
+                    FuzzyUtils.SIM_THRESHOLD_PROP_NAME, StartFeedStatement.WAIT_FOR_COMPLETION,
                     FeedActivityDetails.FEED_POLICY_NAME, FeedActivityDetails.COLLECT_LOCATIONS, "inline_with",
                     "hash_merge", "output-record-type");
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/FeedEventsListener.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/FeedEventsListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/FeedEventsListener.java
index 38e8a21..c0ce6ec 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/FeedEventsListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/FeedEventsListener.java
@@ -67,8 +67,8 @@ public class FeedEventsListener extends ActiveEntityEventsListener {
     @Override
     public synchronized void remove(Dataset dataset) throws HyracksDataException {
         super.remove(dataset);
-        feedConnections.removeIf(o -> o.getDataverseName().equals(dataset.getDataverseName())
-                && o.getDatasetName().equals(dataset.getDatasetName()));
+        feedConnections.removeIf(o -> o.getDataverseName().equals(dataset.getDataverseName()) && o.getDatasetName()
+                .equals(dataset.getDatasetName()));
     }
 
     public synchronized void addFeedConnection(FeedConnection feedConnection) {
@@ -82,12 +82,8 @@ public class FeedEventsListener extends ActiveEntityEventsListener {
     @Override
     protected void doStart(MetadataProvider mdProvider) throws HyracksDataException {
         try {
-            ILangCompilationProvider compilationProvider = new AqlCompilationProvider();
-            IStorageComponentProvider storageComponentProvider = new StorageComponentProvider();
-            DefaultStatementExecutorFactory statementExecutorFactory = new DefaultStatementExecutorFactory();
-            Pair<JobSpecification, AlgebricksAbsolutePartitionConstraint> jobInfo = FeedOperations.buildStartFeedJob(
-                    ((QueryTranslator) statementExecutor).getSessionOutput(), mdProvider, feed, feedConnections,
-                    compilationProvider, storageComponentProvider, statementExecutorFactory, hcc);
+            Pair<JobSpecification, AlgebricksAbsolutePartitionConstraint> jobInfo =
+                    FeedOperations.buildStartFeedJob(mdProvider, feed, feedConnections, statementExecutor, hcc);
             JobSpecification feedJob = jobInfo.getLeft();
             WaitForStateSubscriber eventSubscriber = new WaitForStateSubscriber(this, EnumSet.of(ActivityState.RUNNING,
                     ActivityState.TEMPORARILY_FAILED, ActivityState.PERMANENTLY_FAILED));
@@ -119,8 +115,8 @@ public class FeedEventsListener extends ActiveEntityEventsListener {
             // Construct ActiveMessage
             for (int i = 0; i < getLocations().getLocations().length; i++) {
                 String intakeLocation = getLocations().getLocations()[i];
-                FeedOperations.SendStopMessageToNode(metadataProvider.getApplicationContext(), entityId, intakeLocation,
-                        i);
+                FeedOperations
+                        .SendStopMessageToNode(metadataProvider.getApplicationContext(), entityId, intakeLocation, i);
             }
             eventSubscriber.sync();
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 6b4483c..d3be23c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -122,6 +122,7 @@ import org.apache.asterix.lang.common.statement.TypeDecl;
 import org.apache.asterix.lang.common.statement.TypeDropStatement;
 import org.apache.asterix.lang.common.statement.WriteStatement;
 import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.sqlpp.rewrites.SqlppRewriterFactory;
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.MetadataTransactionContext;
@@ -1669,7 +1670,8 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                 throw new AlgebricksException("There is no dataverse with this name " + dataverse + ".");
             }
             Function function = new Function(dataverse, functionName, cfs.getaAterixFunction().getArity(),
-                    cfs.getParamList(), Function.RETURNTYPE_VOID, cfs.getFunctionBody(), Function.LANGUAGE_AQL,
+                    cfs.getParamList(), Function.RETURNTYPE_VOID, cfs.getFunctionBody(),
+                    rewriterFactory instanceof SqlppRewriterFactory ? Function.LANGUAGE_SQLPP : Function.LANGUAGE_AQL,
                     FunctionKind.SCALAR.toString());
             MetadataManager.INSTANCE.addFunction(mdTxnCtx, function);
 
@@ -2156,6 +2158,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         String policyName = cfs.getPolicy();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        // TODO: Check whether we are connecting a change feed to a non-meta dataset
         // Check whether feed is alive
         ActiveNotificationHandler activeEventHandler =
                 (ActiveNotificationHandler) appCtx.getActiveNotificationHandler();
@@ -2186,7 +2189,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                 throw new AlgebricksException("Feed" + feedName + " is already connected dataset " + datasetName);
             }
             fc = new FeedConnection(dataverseName, feedName, datasetName, appliedFunctions, policyName,
-                    outputType.toString());
+                    outputType.getTypeName());
             MetadataManager.INSTANCE.addFeedConnection(metadataProvider.getMetadataTxnContext(), fc);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             if (listener != null) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
index cc95770..2b9386e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
@@ -20,6 +20,7 @@ package org.apache.asterix.utils;
 
 import java.rmi.RemoteException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -33,32 +34,53 @@ import org.apache.asterix.active.EntityId;
 import org.apache.asterix.active.message.ActiveManagerMessage;
 import org.apache.asterix.active.message.ActiveManagerMessage.Kind;
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
+import org.apache.asterix.app.translator.QueryTranslator;
 import org.apache.asterix.common.cluster.IClusterStateManager;
-import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
 import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
-import org.apache.asterix.external.feed.management.FeedConnectionRequest;
 import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
 import org.apache.asterix.external.feed.watch.FeedActivityDetails;
 import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
 import org.apache.asterix.external.operators.FeedIntakeOperatorDescriptor;
 import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
 import org.apache.asterix.external.operators.FeedMetaOperatorDescriptor;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.FeedConstants;
 import org.apache.asterix.external.util.FeedUtils;
 import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
-import org.apache.asterix.lang.aql.statement.SubscribeFeedStatement;
+import org.apache.asterix.file.StorageComponentProvider;
+import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.statement.DataverseDecl;
+import org.apache.asterix.lang.common.clause.LetClause;
+import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
+import org.apache.asterix.lang.common.expression.VariableExpr;
+import org.apache.asterix.lang.common.literal.IntegerLiteral;
+import org.apache.asterix.lang.common.literal.StringLiteral;
+import org.apache.asterix.lang.common.statement.InsertStatement;
+import org.apache.asterix.lang.common.statement.Query;
+import org.apache.asterix.lang.common.statement.UpsertStatement;
 import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.lang.sqlpp.clause.FromClause;
+import org.apache.asterix.lang.sqlpp.clause.FromTerm;
+import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
+import org.apache.asterix.lang.sqlpp.clause.SelectClause;
+import org.apache.asterix.lang.sqlpp.clause.SelectElement;
+import org.apache.asterix.lang.sqlpp.clause.SelectSetOperation;
+import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.struct.SetOperationInput;
+import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.metadata.entities.Feed;
 import org.apache.asterix.metadata.entities.FeedConnection;
@@ -109,6 +131,8 @@ import org.apache.hyracks.dataflow.std.misc.ReplicateOperatorDescriptor;
  */
 public class FeedOperations {
 
+    public static final String FEED_DATAFLOW_INTERMEIDATE_VAL_PREFIX = "val";
+
     private FeedOperations() {
     }
 
@@ -154,30 +178,76 @@ public class FeedOperations {
         return spec;
     }
 
-    private static JobSpecification getConnectionJob(SessionOutput sessionOutput, MetadataProvider metadataProvider,
-            FeedConnection feedConnection, String[] locations, ILangCompilationProvider compilationProvider,
-            IStorageComponentProvider storageComponentProvider, DefaultStatementExecutorFactory qtFactory,
-            IHyracksClientConnection hcc) throws AlgebricksException, RemoteException, ACIDException {
-        DataverseDecl dataverseDecl = new DataverseDecl(new Identifier(feedConnection.getDataverseName()));
-        FeedConnectionRequest fcr =
-                new FeedConnectionRequest(FeedRuntimeType.INTAKE, feedConnection.getAppliedFunctions(),
-                        feedConnection.getDatasetName(), feedConnection.getPolicyName(), feedConnection.getFeedId());
-        SubscribeFeedStatement subscribeStmt = new SubscribeFeedStatement(locations, fcr);
-        subscribeStmt.initialize(metadataProvider.getMetadataTxnContext());
-        List<Statement> statements = new ArrayList<>();
-        statements.add(dataverseDecl);
-        statements.add(subscribeStmt);
-        IStatementExecutor translator = qtFactory.create(metadataProvider.getApplicationContext(), statements,
-                sessionOutput, compilationProvider, storageComponentProvider);
-        // configure the metadata provider
-        metadataProvider.getConfig().put(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, "" + Boolean.TRUE);
-        metadataProvider.getConfig().put(FeedActivityDetails.FEED_POLICY_NAME, "" + subscribeStmt.getPolicy());
-        metadataProvider.getConfig().put(FeedActivityDetails.COLLECT_LOCATIONS,
-                StringUtils.join(subscribeStmt.getLocations(), ','));
+    private static List<Expression> addArgs(Object... args) {
+        List<Expression> argExprs = new ArrayList<>();
+        for (Object arg : args) {
+            if (arg instanceof Integer) {
+                argExprs.add(new LiteralExpr(new IntegerLiteral((Integer) arg)));
+            } else if (arg instanceof String) {
+                argExprs.add(new LiteralExpr(new StringLiteral((String) arg)));
+            } else if (arg instanceof VariableExpr) {
+                argExprs.add((VariableExpr) arg);
+            }
+        }
+        return argExprs;
+    }
 
-        CompiledStatements.CompiledSubscribeFeedStatement csfs = new CompiledStatements.CompiledSubscribeFeedStatement(
-                subscribeStmt.getSubscriptionRequest(), subscribeStmt.getVarCounter());
-        return translator.rewriteCompileQuery(hcc, metadataProvider, subscribeStmt.getQuery(), csfs);
+    private static Query makeConnectionQuery(FeedConnection feedConnection) {
+        // Construct from clause
+        VarIdentifier fromVarId = SqlppVariableUtil.toInternalVariableIdentifier(feedConnection.getFeedName());
+        VariableExpr fromTermLeftExpr = new VariableExpr(fromVarId);
+        // TODO: remove target feedid from args list (xikui)
+        // TODO: Get rid of this INTAKE
+        List<Expression> exprList =
+                addArgs(feedConnection.getDataverseName(), feedConnection.getFeedId().getEntityName(),
+                        feedConnection.getFeedId().getEntityName(), FeedRuntimeType.INTAKE.toString(),
+                        feedConnection.getDatasetName(), feedConnection.getOutputType());
+        CallExpr datasrouceCallFunction = new CallExpr(FeedConstants.FEED_COLLECT_FUN_SIGNATURE, exprList);
+        FromTerm fromterm = new FromTerm(datasrouceCallFunction, fromTermLeftExpr, null, null);
+        FromClause fromClause = new FromClause(Arrays.asList(fromterm));
+        // TODO: This can be the place to add select predicate for ingestion
+        // Attaching functions
+        int varIdx = 1;
+        VariableExpr previousVarExpr = fromTermLeftExpr;
+        ArrayList<LetClause> letClauses = new ArrayList<>();
+        for (FunctionSignature funcSig : feedConnection.getAppliedFunctions()) {
+            VarIdentifier intermediateVar = SqlppVariableUtil
+                    .toInternalVariableIdentifier(FEED_DATAFLOW_INTERMEIDATE_VAL_PREFIX + String.valueOf(varIdx));
+            VariableExpr intermediateVarExpr = new VariableExpr(intermediateVar);
+            CallExpr functionCallExpr = new CallExpr(funcSig, addArgs(previousVarExpr));
+            previousVarExpr = intermediateVarExpr;
+            LetClause letClause = new LetClause(intermediateVarExpr, functionCallExpr);
+            letClauses.add(letClause);
+            varIdx++;
+        }
+        // Constructing select clause
+        SelectElement selectElement = new SelectElement(previousVarExpr);
+        SelectClause selectClause = new SelectClause(selectElement, null, false);
+        SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, letClauses, null, null, null, null);
+        SelectSetOperation selectSetOperation = new SelectSetOperation(new SetOperationInput(selectBlock, null), null);
+        SelectExpression body = new SelectExpression(null, selectSetOperation, null, null, true);
+        Query query = new Query(false, true, body, 0);
+        return query;
+    }
+
+    private static JobSpecification getConnectionJob(MetadataProvider metadataProvider, FeedConnection feedConn,
+            IStatementExecutor statementExecutor, IHyracksClientConnection hcc, Boolean insertFeed)
+            throws AlgebricksException, RemoteException, ACIDException {
+        metadataProvider.getConfig().put(FeedActivityDetails.FEED_POLICY_NAME, feedConn.getPolicyName());
+        Query feedConnQuery = makeConnectionQuery(feedConn);
+        CompiledStatements.ICompiledDmlStatement clfrqs;
+        if (insertFeed) {
+            InsertStatement stmtUpsert = new InsertStatement(new Identifier(feedConn.getDataverseName()),
+                    new Identifier(feedConn.getDatasetName()), feedConnQuery, -1, null, null);
+            clfrqs = new CompiledStatements.CompiledInsertStatement(feedConn.getDataverseName(),
+                    feedConn.getDatasetName(), feedConnQuery, stmtUpsert.getVarCounter(), null, null);
+        } else {
+            UpsertStatement stmtUpsert = new UpsertStatement(new Identifier(feedConn.getDataverseName()),
+                    new Identifier(feedConn.getDatasetName()), feedConnQuery, -1, null, null);
+            clfrqs = new CompiledStatements.CompiledUpsertStatement(feedConn.getDataverseName(),
+                    feedConn.getDatasetName(), feedConnQuery, stmtUpsert.getVarCounter(), null, null);
+        }
+        return statementExecutor.rewriteCompileQuery(hcc, metadataProvider, feedConnQuery, clfrqs);
     }
 
     private static JobSpecification combineIntakeCollectJobs(MetadataProvider metadataProvider, Feed feed,
@@ -220,9 +290,9 @@ public class FeedOperations {
             String datasetName = feedConnections.get(iter1).getDatasetName();
             FeedConnectionId feedConnectionId = new FeedConnectionId(ingestionOp.getEntityId(), datasetName);
 
-            FeedPolicyEntity feedPolicyEntity =
-                    FeedMetadataUtil.validateIfPolicyExists(curFeedConnection.getDataverseName(),
-                            curFeedConnection.getPolicyName(), metadataProvider.getMetadataTxnContext());
+            FeedPolicyEntity feedPolicyEntity = FeedMetadataUtil
+                    .validateIfPolicyExists(curFeedConnection.getDataverseName(), curFeedConnection.getPolicyName(),
+                            metadataProvider.getMetadataTxnContext());
 
             for (Map.Entry<OperatorDescriptorId, IOperatorDescriptor> entry : operatorsMap.entrySet()) {
                 IOperatorDescriptor opDesc = entry.getValue();
@@ -358,24 +428,39 @@ public class FeedOperations {
         return jobSpec;
     }
 
+    private static IStatementExecutor getSQLPPTranslator(MetadataProvider metadataProvider,
+            SessionOutput sessionOutput) {
+        List<Statement> stmts = new ArrayList<>();
+        DefaultStatementExecutorFactory qtFactory = new DefaultStatementExecutorFactory();
+        IStatementExecutor translator = qtFactory
+                .create(metadataProvider.getApplicationContext(), stmts, sessionOutput, new SqlppCompilationProvider(),
+                        new StorageComponentProvider());
+        return translator;
+    }
+
     public static Pair<JobSpecification, AlgebricksAbsolutePartitionConstraint> buildStartFeedJob(
-            SessionOutput sessionOutput, MetadataProvider metadataProvider, Feed feed,
-            List<FeedConnection> feedConnections, ILangCompilationProvider compilationProvider,
-            IStorageComponentProvider storageComponentProvider, DefaultStatementExecutorFactory qtFactory,
-            IHyracksClientConnection hcc) throws Exception {
+            MetadataProvider metadataProvider, Feed feed, List<FeedConnection> feedConnections,
+            IStatementExecutor statementExecutor, IHyracksClientConnection hcc) throws Exception {
         FeedPolicyAccessor fpa = new FeedPolicyAccessor(new HashMap<>());
-        // TODO: Change the default Datasource to use all possible partitions
         Pair<JobSpecification, IAdapterFactory> intakeInfo = buildFeedIntakeJobSpec(feed, metadataProvider, fpa);
-        //TODO: Add feed policy accessor
         List<JobSpecification> jobsList = new ArrayList<>();
+        // TODO: Figure out a better way to handle insert/upsert per conn instead of per feed
+        Boolean insertFeed = ExternalDataUtils.isInsertFeed(feed.getAdapterConfiguration());
         // Construct the ingestion Job
         JobSpecification intakeJob = intakeInfo.getLeft();
         IAdapterFactory ingestionAdaptorFactory = intakeInfo.getRight();
         String[] ingestionLocations = ingestionAdaptorFactory.getPartitionConstraint().getLocations();
+        // Add metadata configs
+        metadataProvider.getConfig().put(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, Boolean.TRUE.toString());
+        metadataProvider.getConfig()
+                .put(FeedActivityDetails.COLLECT_LOCATIONS, StringUtils.join(ingestionLocations, ','));
+        // TODO: Once we deprecated AQL, this extra queryTranslator can be removed.
+        IStatementExecutor translator =
+                getSQLPPTranslator(metadataProvider, ((QueryTranslator) statementExecutor).getSessionOutput());
         // Add connection job
         for (FeedConnection feedConnection : feedConnections) {
-            JobSpecification connectionJob = getConnectionJob(sessionOutput, metadataProvider, feedConnection,
-                    ingestionLocations, compilationProvider, storageComponentProvider, qtFactory, hcc);
+            JobSpecification connectionJob = getConnectionJob(metadataProvider, feedConnection, translator, hcc,
+                    insertFeed);
             jobsList.add(connectionJob);
         }
         return Pair.of(combineIntakeCollectJobs(metadataProvider, feed, intakeJob, jobsList, feedConnections,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/cross-dataverse/cross-dv15/cross-dv15.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/cross-dataverse/cross-dv15/cross-dv15.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/cross-dataverse/cross-dv15/cross-dv15.3.query.aql
index 1d99878..2dc595b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/cross-dataverse/cross-dv15/cross-dv15.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/cross-dataverse/cross-dv15/cross-dv15.3.query.aql
@@ -30,6 +30,5 @@ return {
 "DataverseName": $l.DataverseName,
 "Name": $l.Name,
 "Arity": $l.Arity,
-"ReturnType": $l.ReturnType,
-"Language": $l.Language
+"ReturnType": $l.ReturnType
 };

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.1.ddl.aql
deleted file mode 100644
index 631c3fb..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.1.ddl.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed and apply two functions in the
- * workflow. The output of the first function can be used in
- * the second function. The function parameter can have any
- * name.
- * Expected Res : Success
- * Date         : 29th Mar 2017
- */
-drop dataverse experiments if exists;
-create dataverse experiments;
-use dataverse experiments;
-
-create type TwitterUser if not exists as open{
-    screen-name: string,
-    friends_count: int32,
-    name: string,
-    followers_count: int32
-};
-
-create dataset TwitterUsers(TwitterUser) primary key screen-name;
-
-create function test_func0($xyz) {
-    let $tty1 := if ($xyz.followers_count > 25000) then {"popularity":"Good!"} else {"popularity":"Bad!"}
-    return object_merge($tty1, $xyz)
-}
-
-create function test_func1($anyname) {
-    let $tty2 := if ($anyname.popularity = "Good!") then {"true_popularity":"Indeed Good!"} else {"true_popularity":"Indeed Bad!"}
-    return object_merge($tty2, $anyname)
-}
-
-create feed UserFeed using socket_adapter
-(
-    ("sockets"="127.0.0.1:10001"),
-    ("address-type"="IP"),
-    ("type-name"="TwitterUser"),
-    ("format"="adm"),
-    ("upsert-feed"="true")
-);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.2.update.aql
deleted file mode 100644
index a5933a5..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.2.update.aql
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed and apply two functions in the
- * workflow. The output of the first function can be used in
- * the second function. The function parameter can have any
- * name.
- * Expected Res : Success
- * Date         : 29th Mar 2017
- */
-use dataverse experiments;
-set wait-for-completion-feed "false";
-
-connect feed UserFeed to dataset TwitterUsers apply function test_func0,test_func1;
-
-start feed UserFeed;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.3.server.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.3.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.3.server.aql
deleted file mode 100644
index eacf623..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.3.server.aql
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed and apply two functions in the
- * workflow. The output of the first function can be used in
- * the second function. The function parameter can have any
- * name.
- * Expected Res : Success
- * Date         : 29th Mar 2017
- */
-start client 10001 file-client 127.0.0.1 ../asterix-app/data/tinysocial/twu.adm 500 50 1000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.4.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.4.sleep.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.4.sleep.aql
deleted file mode 100644
index dc5dae0..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.4.sleep.aql
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed and apply two functions in the
- * workflow. The output of the first function can be used in
- * the second function. The function parameter can have any
- * name.
- * Expected Res : Success
- * Date         : 29th Mar 2017
- */
-2000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.5.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.5.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.5.update.aql
deleted file mode 100644
index dcf2278..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.5.update.aql
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed and apply two functions in the
- * workflow. The output of the first function can be used in
- * the second function. The function parameter can have any
- * name.
- * Expected Res : Success
- * Date         : 29th Mar 2017
- */
-use dataverse experiments;
-stop feed UserFeed;
-disconnect feed UserFeed from dataset TwitterUsers;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.6.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.6.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.6.query.aql
deleted file mode 100644
index 1a06334..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.6.query.aql
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed and apply two functions in the
- * workflow. The output of the first function can be used in
- * the second function. The function parameter can have any
- * name.
- * Expected Res : Success
- * Date         : 29th Mar 2017
- */
-use dataverse experiments;
-
-for $x in dataset TwitterUsers
-order by $x.screen-name
-return $x.true_popularity;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.7.server.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.7.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.7.server.aql
deleted file mode 100644
index 4ba1c81..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.7.server.aql
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed and apply two functions in the
- * workflow. The output of the first function can be used in
- * the second function. The function parameter can have any
- * name.
- * Expected Res : Success
- * Date         : 29th Mar 2017
- */
-
-stop 10001
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.8.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.8.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.8.ddl.aql
deleted file mode 100644
index 7722945..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-aql-function/connect-feed-with-aql-function.8.ddl.aql
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed and apply two functions in the
- * workflow. The output of the first function can be used in
- * the second function. The function parameter can have any
- * name.
- * Expected Res : Success
- * Date         : 29th Mar 2017
- */
-use dataverse experiments;
-drop dataverse experiments;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-function/connect-feed-with-function.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-function/connect-feed-with-function.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-function/connect-feed-with-function.1.ddl.aql
new file mode 100644
index 0000000..631c3fb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-function/connect-feed-with-function.1.ddl.aql
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed and apply two functions in the
+ * workflow. The output of the first function can be used in
+ * the second function. The function parameter can have any
+ * name.
+ * Expected Res : Success
+ * Date         : 29th Mar 2017
+ */
+drop dataverse experiments if exists;
+create dataverse experiments;
+use dataverse experiments;
+
+create type TwitterUser if not exists as open{
+    screen-name: string,
+    friends_count: int32,
+    name: string,
+    followers_count: int32
+};
+
+create dataset TwitterUsers(TwitterUser) primary key screen-name;
+
+create function test_func0($xyz) {
+    let $tty1 := if ($xyz.followers_count > 25000) then {"popularity":"Good!"} else {"popularity":"Bad!"}
+    return object_merge($tty1, $xyz)
+}
+
+create function test_func1($anyname) {
+    let $tty2 := if ($anyname.popularity = "Good!") then {"true_popularity":"Indeed Good!"} else {"true_popularity":"Indeed Bad!"}
+    return object_merge($tty2, $anyname)
+}
+
+create feed UserFeed using socket_adapter
+(
+    ("sockets"="127.0.0.1:10001"),
+    ("address-type"="IP"),
+    ("type-name"="TwitterUser"),
+    ("format"="adm"),
+    ("upsert-feed"="true")
+);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-function/connect-feed-with-function.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-function/connect-feed-with-function.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-function/connect-feed-with-function.2.update.aql
new file mode 100644
index 0000000..a5933a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-function/connect-feed-with-function.2.update.aql
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed and apply two functions in the
+ * workflow. The output of the first function can be used in
+ * the second function. The function parameter can have any
+ * name.
+ * Expected Res : Success
+ * Date         : 29th Mar 2017
+ */
+use dataverse experiments;
+set wait-for-completion-feed "false";
+
+connect feed UserFeed to dataset TwitterUsers apply function test_func0,test_func1;
+
+start feed UserFeed;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-function/connect-feed-with-function.3.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-function/connect-feed-with-function.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-function/connect-feed-with-function.3.ddl.aql
new file mode 100644
index 0000000..46056b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/connect-feed-with-function/connect-feed-with-function.3.ddl.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed and apply two functions in the
+ * workflow. The output of the first function can be used in
+ * the second function. The function parameter can have any
+ * name.
+ * Expected Res : Success
+ * Date         : 29th Mar 2017
+ */
+drop dataverse experiments;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.1.ddl.aql
new file mode 100644
index 0000000..8dba0fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.1.ddl.aql
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create a feed with insert option. Push record twice
+ * with minor changes. The updated record will not be updated
+ * Expected Res : Success
+ * Date         : 8th Oct 2017
+ */
+drop dataverse experiments if exists;
+create dataverse experiments;
+use dataverse experiments;
+
+create type TwitterUser if not exists as open{
+    screen-name: string,
+    friends_count: int32,
+    name: string,
+    followers_count: int32
+};
+
+create dataset TwitterUsers(TwitterUser) primary key screen-name;
+
+create feed UserFeed using socket_adapter
+(
+    ("sockets"="127.0.0.1:10001"),
+    ("address-type"="IP"),
+    ("type-name"="TwitterUser"),
+    ("format"="adm"),
+    ("insert-feed"="true")
+);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.2.update.aql
new file mode 100644
index 0000000..2a9c11c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.2.update.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use dataverse experiments;
+set wait-for-completion-feed "false";
+
+connect feed UserFeed to dataset TwitterUsers;
+
+start feed UserFeed;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.3.server.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.3.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.3.server.aql
new file mode 100644
index 0000000..873acae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.3.server.aql
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+start client 10001 file-client 127.0.0.1 ../asterix-app/data/tinysocial/twu_update.adm 500 50 1000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.4.sleep.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.4.sleep.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.4.sleep.aql
new file mode 100644
index 0000000..5af9639
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.4.sleep.aql
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+1000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.5.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.5.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.5.update.aql
new file mode 100644
index 0000000..4904a5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.5.update.aql
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use dataverse experiments;
+stop feed UserFeed;
+disconnect feed UserFeed from dataset TwitterUsers;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.6.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.6.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.6.query.aql
new file mode 100644
index 0000000..6085520
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.6.query.aql
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use dataverse experiments;
+
+for $x in dataset TwitterUsers
+order by $x.screen-name
+return $x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.7.server.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.7.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.7.server.aql
new file mode 100644
index 0000000..f9134bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.7.server.aql
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+stop 10001
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.8.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.8.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.8.ddl.aql
new file mode 100644
index 0000000..94276a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/insert-feed/insert-feed.8.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use dataverse experiments;
+drop dataverse experiments;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.1.ddl.aql
deleted file mode 100644
index 14b857f..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.1.ddl.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-drop dataverse experiments if exists;
-create dataverse experiments;
-use dataverse experiments;
-
-create type TwitterUser if not exists as open{
-    screen-name: string,
-    friends_count: int32,
-    name: string,
-    followers_count: int32
-};
-
-create dataset TwitterUsers(TwitterUser) primary key screen-name;
-
-create feed UserFeed using socket_adapter
-(
-    ("sockets"="127.0.0.1:10001"),
-    ("address-type"="IP"),
-    ("type-name"="TwitterUser"),
-    ("format"="adm"),
-    ("upsert-feed"="true")
-);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.2.update.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.2.update.aql
deleted file mode 100644
index bbf20dd..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.2.update.aql
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-use dataverse experiments;
-set wait-for-completion-feed "false";
-
-connect feed UserFeed to dataset TwitterUsers;
-
-start feed UserFeed;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7e76a079/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.3.server.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.3.server.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.3.server.aql
deleted file mode 100644
index 752ef6e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/feeds/upsert-feed/upsert-feed.3.server.aql
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description  : Create a feed with upsert option. Push record twice
- * with minor changes. The updated record will not cause duplicate
- * key exception and the data will be updated.
- * Expected Res : Success
- * Date         : 13th Aug 2016
- */
-start client 10001 file-client 127.0.0.1 ../asterix-app/data/tinysocial/twu_update.adm 500 50 1000
\ No newline at end of file