You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Xikui Wang (Code Review)" <do...@asterixdb.incubator.apache.org> on 2018/12/11 22:12:50 UTC

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Xikui Wang has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/3072

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................

[ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

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

LET is not necessary for applying functions to data feeds. We could
incline the function calls when constructing the pipeline query.

Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.3.lib.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.4.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/udf_filter_on_feed/udf_filter_on_feed.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
A asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFactory.java
A asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFunction.java
M asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
11 files changed, 251 insertions(+), 15 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/72/3072/1

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 593d7ce..1049e66 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
@@ -184,8 +184,8 @@
                 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);
+            } else if (arg instanceof Expression) {
+                argExprs.add((Expression) arg);
             }
         }
         return argExprs;
@@ -216,25 +216,28 @@
             whereClause = new WhereClause(whereClauseQuery.getBody());
         }
 
-        // 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++;
+        Expression previousVarExpr = fromTermLeftExpr;
+        for (FunctionSignature functionSignature : feedConnection.getAppliedFunctions()) {
+            CallExpr functionCallExpr = new CallExpr(functionSignature, addArgs(previousVarExpr));
+            previousVarExpr = functionCallExpr;
         }
+        //        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, whereClause, null, null, null);
+        SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, null, whereClause, 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);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.1.ddl.sqlpp
new file mode 100644
index 0000000..3f3495c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type InputRecordType as closed {
+  id:int64,
+  fname:string,
+  lname:string,
+  age:int64,
+  dept:string
+};
+
+create type DetectResultType as {
+  id: int64,
+    sensitive: boolean
+};
+
+create dataset Results(DetectResultType) primary key id;
+
+create feed EmployeeFeed with {
+  "adapter-name" : "localfs",
+  "path" : "asterix_nc1://data/names.adm",
+  "type-name" : "InputRecordType",
+  "format" : "delimited-text",
+  "delimiter" : "|",
+  "insert-feed": "true"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.2.update.sqlpp
new file mode 100644
index 0000000..f58048f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.2.update.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.
+ */
+
+use test;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.3.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.3.lib.sqlpp
new file mode 100644
index 0000000..dbdfe16
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.3.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 test testlib target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.4.update.sqlpp
new file mode 100644
index 0000000..cdca69b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.4.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 test;
+
+connect feed EmployeeFeed to dataset Results APPLY FUNCTION testlib#fnameDetector WHERE testlib#wordDetector(fname) = TRUE;
+
+START FEED EmployeeFeed;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.5.query.sqlpp
new file mode 100644
index 0000000..9626f36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.5.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 test;
+
+select count(*) from Results;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/udf_filter_on_feed/udf_filter_on_feed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/udf_filter_on_feed/udf_filter_on_feed.1.adm
new file mode 100644
index 0000000..e440e5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/udf_filter_on_feed/udf_filter_on_feed.1.adm
@@ -0,0 +1 @@
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
index a306a90..13de599 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
@@ -9,3 +9,4 @@
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#parseTweet", "Arity": "1", "Params": [ "TweetInputType" ], "ReturnType": "TweetOutputType", "Definition": "org.apache.asterix.external.library.ParseTweetFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#toUpper", "Arity": "1", "Params": [ "TextType" ], "ReturnType": "TextType", "Definition": "org.apache.asterix.external.library.UpperCaseFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#typeValidation", "Arity": "11", "Params": [ "AINT32", "AFLOAT", "ASTRING", "ADouble", "ABoolean", "APoint", "ADate", "ADatetime", "ALine", "ACircle", "ARectangle" ], "ReturnType": "AString", "Definition": "org.apache.asterix.external.library.TypeValidationFunctionFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
+{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#wordDetector", "Arity": "1", "Params": [ "ASTRING" ], "ReturnType": "ABOOLEAN", "Definition": "org.apache.asterix.external.library.WordInListFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFactory.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFactory.java
new file mode 100644
index 0000000..6f87c10
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFactory.java
@@ -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.
+ */
+
+package org.apache.asterix.external.library;
+
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionFactory;
+
+public class WordInListFactory implements IFunctionFactory {
+
+    @Override
+    public IExternalScalarFunction getExternalFunction() {
+        return new WordInListFunction();
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFunction.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFunction.java
new file mode 100644
index 0000000..8295fb2
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFunction.java
@@ -0,0 +1,60 @@
+/*
+ * 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.external.library;
+
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionHelper;
+import org.apache.asterix.external.library.java.base.JBoolean;
+import org.apache.asterix.external.library.java.base.JString;
+
+public class WordInListFunction implements IExternalScalarFunction {
+
+    private ArrayList<String> keywordsList;
+    private String dictPath;
+    private List<String> functionParameters;
+
+    @Override
+    public void evaluate(IFunctionHelper functionHelper) throws Exception {
+        JString input = (JString) functionHelper.getArgument(0);
+        JBoolean output = (JBoolean) functionHelper.getResultObject();
+        String fieldValue = input.getValue();
+        boolean contains = keywordsList.contains(fieldValue);
+        output.setValue(contains);
+        functionHelper.setResult(output);
+    }
+
+    @Override
+    public void initialize(IFunctionHelper functionHelper) throws Exception {
+        keywordsList = new ArrayList<>();
+        functionParameters = functionHelper.getParameters();
+        dictPath = functionParameters.get(0);
+        Files.lines(Paths.get(dictPath)).forEach(keyword -> keywordsList.add(keyword));
+    }
+
+    @Override
+    public void deinitialize() {
+        // no op
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml b/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
index 45cdc35..acbc003 100644
--- a/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
+++ b/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
@@ -20,6 +20,14 @@
   <language>JAVA</language>
   <libraryFunctions>
     <libraryFunction>
+      <name>wordDetector</name>
+      <function_type>SCALAR</function_type>
+      <argument_type>ASTRING</argument_type>
+      <return_type>ABOOLEAN</return_type>
+      <definition>org.apache.asterix.external.library.WordInListFactory</definition>
+      <parameters>data/external_function/KeywordsDetector_List1.txt</parameters>
+    </libraryFunction>
+    <libraryFunction>
       <name>fnameDetector</name>
       <function_type>SCALAR</function_type>
       <argument_type>InputRecordType</argument_type>

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/3856/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5210/ (14/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://goo.gl/5PNgQi : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2839/ (9/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/fytNhr : UNSTABLE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://goo.gl/zxXPCs : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/10311/ (4/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/246/ (12/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4877/ (12/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5216/ (9/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4740/ (1/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8778/ (5/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5172/ (13/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4836/ (2/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/34/ (9/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/53/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2880/ (8/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4741/ (10/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8779/ (2/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2: Contrib-2

BAD Compatibility Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/3845/ : FAILURE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4796/ (6/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7657/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/3845/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/18xBfx : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Dmitry Lychagin has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5215/ (2/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2: -Contrib

Analytics Compatibility Compilation Successful
https://goo.gl/z4SQzT : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5440/ (10/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4835/ (12/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Xikui Wang (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/3072

to look at the new patch set (#2).

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................

[ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

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

LET is not necessary for applying functions to data feeds. We could
incline the function calls when constructing the pipeline query.

Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.3.lib.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.4.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/udf_filter_on_feed/udf_filter_on_feed.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
A asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFactory.java
A asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFunction.java
M asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
11 files changed, 240 insertions(+), 16 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/72/3072/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/10352/ (5/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/25/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7657/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/252/ (1/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/247/ (11/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4830/ (5/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4834/ (11/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Xikui Wang (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/3072

to look at the new patch set (#3).

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................

[ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

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

LET is not necessary for applying functions to data feeds. We could
inline the function calls when constructing the pipeline query.

Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.3.lib.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.4.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/udf_filter_on_feed/udf_filter_on_feed.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
A asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFactory.java
A asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFunction.java
M asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
11 files changed, 240 insertions(+), 16 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/72/3072/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/288/ (6/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4782/ (1/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/5051/ (4/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7694/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2840/ (3/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5400/ (7/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7694/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Xikui Wang (Code Review)" <do...@asterixdb.incubator.apache.org>.
Xikui Wang has submitted this change and it was merged.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


[ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

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

LET is not necessary for applying functions to data feeds. We could
inline the function calls when constructing the pipeline query.

Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3072
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.1.ddl.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.2.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.3.lib.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.4.update.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/udf_filter_on_feed/udf_filter_on_feed.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
A asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFactory.java
A asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFunction.java
M asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
11 files changed, 240 insertions(+), 16 deletions(-)

Approvals:
  Anon. E. Moose #1000171: 
  Jenkins: Verified; No violations found; ; Verified
  Dmitry Lychagin: Looks good to me, approved



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 593d7ce..d22e929 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
@@ -184,8 +184,8 @@
                 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);
+            } else if (arg instanceof Expression) {
+                argExprs.add((Expression) arg);
             }
         }
         return argExprs;
@@ -216,25 +216,16 @@
             whereClause = new WhereClause(whereClauseQuery.getBody());
         }
 
-        // 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++;
+        Expression previousVarExpr = fromTermLeftExpr;
+        for (FunctionSignature functionSignature : feedConnection.getAppliedFunctions()) {
+            CallExpr functionCallExpr = new CallExpr(functionSignature, addArgs(previousVarExpr));
+            previousVarExpr = functionCallExpr;
         }
         // Constructing select clause
         SelectElement selectElement = new SelectElement(previousVarExpr);
         SelectClause selectClause = new SelectClause(selectElement, null, false);
-        SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, letClauses, whereClause, null, null, null);
+        SelectBlock selectBlock = new SelectBlock(selectClause, fromClause, null, whereClause, 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);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.1.ddl.sqlpp
new file mode 100644
index 0000000..3f3495c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type InputRecordType as closed {
+  id:int64,
+  fname:string,
+  lname:string,
+  age:int64,
+  dept:string
+};
+
+create type DetectResultType as {
+  id: int64,
+    sensitive: boolean
+};
+
+create dataset Results(DetectResultType) primary key id;
+
+create feed EmployeeFeed with {
+  "adapter-name" : "localfs",
+  "path" : "asterix_nc1://data/names.adm",
+  "type-name" : "InputRecordType",
+  "format" : "delimited-text",
+  "delimiter" : "|",
+  "insert-feed": "true"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.2.update.sqlpp
new file mode 100644
index 0000000..f58048f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.2.update.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.
+ */
+
+use test;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.3.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.3.lib.sqlpp
new file mode 100644
index 0000000..dbdfe16
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.3.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 test testlib target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.4.update.sqlpp
new file mode 100644
index 0000000..cdca69b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.4.update.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 test;
+
+connect feed EmployeeFeed to dataset Results APPLY FUNCTION testlib#fnameDetector WHERE testlib#wordDetector(fname) = TRUE;
+
+START FEED EmployeeFeed;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.5.query.sqlpp
new file mode 100644
index 0000000..9626f36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_filter_on_feed/udf_filter_on_feed.5.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 test;
+
+select count(*) from Results;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/udf_filter_on_feed/udf_filter_on_feed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/udf_filter_on_feed/udf_filter_on_feed.1.adm
new file mode 100644
index 0000000..e440e5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/udf_filter_on_feed/udf_filter_on_feed.1.adm
@@ -0,0 +1 @@
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
index a306a90..13de599 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/validate-default-library/validate-default-library.1.adm
@@ -9,3 +9,4 @@
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#parseTweet", "Arity": "1", "Params": [ "TweetInputType" ], "ReturnType": "TweetOutputType", "Definition": "org.apache.asterix.external.library.ParseTweetFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#toUpper", "Arity": "1", "Params": [ "TextType" ], "ReturnType": "TextType", "Definition": "org.apache.asterix.external.library.UpperCaseFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
 { "Function": { "DataverseName": "externallibtest", "Name": "testlib#typeValidation", "Arity": "11", "Params": [ "AINT32", "AFLOAT", "ASTRING", "ADouble", "ABoolean", "APoint", "ADate", "ADatetime", "ALine", "ACircle", "ARectangle" ], "ReturnType": "AString", "Definition": "org.apache.asterix.external.library.TypeValidationFunctionFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
+{ "Function": { "DataverseName": "externallibtest", "Name": "testlib#wordDetector", "Arity": "1", "Params": [ "ASTRING" ], "ReturnType": "ABOOLEAN", "Definition": "org.apache.asterix.external.library.WordInListFactory", "Language": "JAVA", "Kind": "SCALAR", "Dependencies": [ [  ], [  ] ] } }
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFactory.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFactory.java
new file mode 100644
index 0000000..6f87c10
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFactory.java
@@ -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.
+ */
+
+package org.apache.asterix.external.library;
+
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionFactory;
+
+public class WordInListFactory implements IFunctionFactory {
+
+    @Override
+    public IExternalScalarFunction getExternalFunction() {
+        return new WordInListFunction();
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFunction.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFunction.java
new file mode 100644
index 0000000..8295fb2
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/WordInListFunction.java
@@ -0,0 +1,60 @@
+/*
+ * 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.external.library;
+
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.external.api.IExternalScalarFunction;
+import org.apache.asterix.external.api.IFunctionHelper;
+import org.apache.asterix.external.library.java.base.JBoolean;
+import org.apache.asterix.external.library.java.base.JString;
+
+public class WordInListFunction implements IExternalScalarFunction {
+
+    private ArrayList<String> keywordsList;
+    private String dictPath;
+    private List<String> functionParameters;
+
+    @Override
+    public void evaluate(IFunctionHelper functionHelper) throws Exception {
+        JString input = (JString) functionHelper.getArgument(0);
+        JBoolean output = (JBoolean) functionHelper.getResultObject();
+        String fieldValue = input.getValue();
+        boolean contains = keywordsList.contains(fieldValue);
+        output.setValue(contains);
+        functionHelper.setResult(output);
+    }
+
+    @Override
+    public void initialize(IFunctionHelper functionHelper) throws Exception {
+        keywordsList = new ArrayList<>();
+        functionParameters = functionHelper.getParameters();
+        dictPath = functionParameters.get(0);
+        Files.lines(Paths.get(dictPath)).forEach(keyword -> keywordsList.add(keyword));
+    }
+
+    @Override
+    public void deinitialize() {
+        // no op
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml b/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
index 45cdc35..acbc003 100644
--- a/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
+++ b/asterixdb/asterix-external-data/src/test/resources/library_descriptor.xml
@@ -20,6 +20,14 @@
   <language>JAVA</language>
   <libraryFunctions>
     <libraryFunction>
+      <name>wordDetector</name>
+      <function_type>SCALAR</function_type>
+      <argument_type>ASTRING</argument_type>
+      <return_type>ABOOLEAN</return_type>
+      <definition>org.apache.asterix.external.library.WordInListFactory</definition>
+      <parameters>data/external_function/KeywordsDetector_List1.txt</parameters>
+    </libraryFunction>
+    <libraryFunction>
       <name>fnameDetector</name>
       <function_type>SCALAR</function_type>
       <argument_type>InputRecordType</argument_type>

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: merged
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/mqnGHt : UNSTABLE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/10312/ (8/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/281/ (13/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/5091/ (11/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Analytics Compatibility Compilation Successful
https://goo.gl/V3tsmn : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8819/ (7/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/3856/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4870/ (3/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/5050/ (8/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/4795/ (7/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4829/ (10/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/24/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7656/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/251/ (6/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Dmitry Lychagin has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 2:

(1 comment)

BAD tests failed, you probably need to rebase

https://asterix-gerrit.ics.uci.edu/#/c/3072/2//COMMIT_MSG
Commit Message:

PS2, Line 14: incline
should be "inline"


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5254/ (4/14)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5171/ (13/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2495][ING] Avoid LET in applying functions to feeds
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5399/ (3/13)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3072
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I65842f9ac84891b363d7e0a02425258d0df794e7
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Xikui Wang <xk...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No