You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by mh...@apache.org on 2019/09/12 14:25:04 UTC

[asterixdb] 05/06: [NO ISSUE][ING][EXT] Handle filters on meta() fields for feeds

This is an automated email from the ASF dual-hosted git repository.

mhubail pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git

commit 8930f0d7614f43ad52624fd121de45ab27fb281e
Author: Ali Alsuliman <al...@gmail.com>
AuthorDate: Wed Sep 11 11:56:53 2019 -0700

    [NO ISSUE][ING][EXT] Handle filters on meta() fields for feeds
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    This fix is to handle filters on meta() fields where the feed name
    is not supplied as an arguement to the meta() when connecting the feed.
    Both of these statement should work:
    CONNECT FEED ds_feed TO DATASET ds WHERE meta(ds_feed).id LIKE "%WEB%"
    CONNECT FEED ds_feed TO DATASET ds WHERE meta().id LIKE "%WEB%"
    
    Change-Id: I1dbf40525601584efa29eb3adf4ba41062ebe53a
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/3557
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
---
 .../rules/MetaFunctionToMetaVariableRule.java      | 60 ++++++++++++++----
 asterixdb/asterix-app/data/csv/people2.csv         | 14 +++++
 .../change-feed-with-filter-on-meta.1.ddl.sqlpp    | 73 ++++++++++++++++++++++
 ...change-feed-with-filter-on-meta.10.update.sqlpp | 38 +++++++++++
 .../change-feed-with-filter-on-meta.11.query.sqlpp | 22 +++++++
 .../change-feed-with-filter-on-meta.12.query.sqlpp | 22 +++++++
 .../change-feed-with-filter-on-meta.13.query.sqlpp | 22 +++++++
 .../change-feed-with-filter-on-meta.14.query.sqlpp | 22 +++++++
 .../change-feed-with-filter-on-meta.15.query.sqlpp | 22 +++++++
 .../change-feed-with-filter-on-meta.16.query.sqlpp | 22 +++++++
 .../change-feed-with-filter-on-meta.17.query.sqlpp | 22 +++++++
 .../change-feed-with-filter-on-meta.18.ddl.sqlpp   | 20 ++++++
 .../change-feed-with-filter-on-meta.2.update.sqlpp | 38 +++++++++++
 .../change-feed-with-filter-on-meta.3.query.sqlpp  | 22 +++++++
 .../change-feed-with-filter-on-meta.4.query.sqlpp  | 22 +++++++
 .../change-feed-with-filter-on-meta.5.query.sqlpp  | 22 +++++++
 .../change-feed-with-filter-on-meta.6.query.sqlpp  | 22 +++++++
 .../change-feed-with-filter-on-meta.7.query.sqlpp  | 22 +++++++
 .../change-feed-with-filter-on-meta.8.query.sqlpp  | 22 +++++++
 .../change-feed-with-filter-on-meta.9.query.sqlpp  | 22 +++++++
 .../change-feed-with-filter-on-meta.11.adm         |  9 +++
 .../change-feed-with-filter-on-meta.12.adm         |  5 ++
 .../change-feed-with-filter-on-meta.13.adm         |  5 ++
 .../change-feed-with-filter-on-meta.14.adm         |  5 ++
 .../change-feed-with-filter-on-meta.15.adm         |  4 ++
 .../change-feed-with-filter-on-meta.16.adm         |  5 ++
 .../change-feed-with-filter-on-meta.17.adm         |  9 +++
 .../change-feed-with-filter-on-meta.3.adm          |  9 +++
 .../change-feed-with-filter-on-meta.4.adm          |  5 ++
 .../change-feed-with-filter-on-meta.5.adm          |  5 ++
 .../change-feed-with-filter-on-meta.6.adm          |  5 ++
 .../change-feed-with-filter-on-meta.7.adm          |  4 ++
 .../change-feed-with-filter-on-meta.8.adm          |  5 ++
 .../change-feed-with-filter-on-meta.9.adm          |  9 +++
 .../test/resources/runtimets/testsuite_sqlpp.xml   |  5 ++
 35 files changed, 627 insertions(+), 13 deletions(-)

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
index c47acd1..dc756ea 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/MetaFunctionToMetaVariableRule.java
@@ -77,13 +77,13 @@ public class MetaFunctionToMetaVariableRule implements IAlgebraicRewriteRule {
         ILogicalOperator op = opRef.getValue();
 
         // Reaches NTS or ETS.
-        if (op.getInputs().size() == 0) {
+        if (op.getInputs().isEmpty()) {
             return NoOpExpressionReferenceTransform.INSTANCE;
         }
         // Datascan returns an useful transform if the meta part presents in the dataset.
         if (op.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
             DataSourceScanOperator scanOp = (DataSourceScanOperator) op;
-            ILogicalExpressionReferenceTransformWithCondition inputTransfomer = visit(op.getInputs().get(0));
+            ILogicalExpressionReferenceTransformWithCondition inputTransformer = visit(op.getInputs().get(0));
             DataSource dataSource = (DataSource) scanOp.getDataSource();
             List<ILogicalExpressionReferenceTransformWithCondition> transformers = null;
             List<LogicalVariable> allVars = scanOp.getVariables();
@@ -106,26 +106,26 @@ public class MetaFunctionToMetaVariableRule implements IAlgebraicRewriteRule {
                 }
             }
             if (!dataSource.hasMeta() && transformers == null) {
-                return inputTransfomer;
+                return inputTransformer;
             }
             if (metaVar != null) {
                 currentTransformer = new LogicalExpressionReferenceTransform(dataVar, metaVar);
             }
-            if (inputTransfomer.equals(NoOpExpressionReferenceTransform.INSTANCE) && transformers == null) {
+            if (inputTransformer.equals(NoOpExpressionReferenceTransform.INSTANCE) && transformers == null) {
                 return currentTransformer;
-            } else if (inputTransfomer.equals(NoOpExpressionReferenceTransform.INSTANCE)
+            } else if (inputTransformer.equals(NoOpExpressionReferenceTransform.INSTANCE)
                     && currentTransformer == null) {
                 return transformers.get(0);
             } else {
-                // Requires an argument variable to resolve ambiguity.
                 if (transformers == null) {
                     transformers = new ArrayList<>();
                 }
-                if (!inputTransfomer.equals(NoOpExpressionReferenceTransform.INSTANCE)) {
-                    inputTransfomer.setVariableRequired();
-                    transformers.add(inputTransfomer);
+                if (!inputTransformer.equals(NoOpExpressionReferenceTransform.INSTANCE)) {
+                    // require an argument variable to resolve ambiguity when there are 2 or more distinct data sources
+                    inputTransformer.setVariableRequired();
+                    currentTransformer.setVariableRequired();
+                    transformers.add(inputTransformer);
                 }
-                currentTransformer.setVariableRequired();
                 transformers.add(currentTransformer);
                 return new CompositeExpressionReferenceTransform(transformers);
             }
@@ -185,6 +185,17 @@ class NoOpExpressionReferenceTransform implements ILogicalExpressionReferenceTra
 
 }
 
+/**
+ * <pre>
+ * This class replaces meta() references with their corresponding meta record variables. It maintains the data record
+ * variable and meta record variable. The data variable is used to match the data variable inside meta() if supplied.
+ * For example:
+ * If the data source produces 2 records, the data record as $$ds and the meta record as $$7, then any reference to
+ * meta($$ds) will be rewritten as $$7.
+ *
+ * meta($$ds) means "get the meta record of the data source ds".
+ * </pre>
+ */
 class LogicalExpressionReferenceTransform implements ILogicalExpressionReferenceTransformWithCondition {
     private final LogicalVariable dataVar;
     private final LogicalVariable metaVar;
@@ -273,6 +284,15 @@ class CompositeExpressionReferenceTransform implements ILogicalExpressionReferen
     }
 }
 
+/**
+ * <pre>
+ * This class replaces meta-key() references with their corresponding field accessors. It maintains the meta
+ * variable that will replace the meta-key(). Meta-key() acts as a field access of the meta record. For example:
+ * If the meta record variable is $$7, meta-key($$ds, "address.zip") will be rewritten as $$7.address.zip.
+ *
+ * meta-key($$ds, "address.zip") means "access the field address.zip of the meta record of data source ds".
+ * </pre>
+ */
 class MetaKeyToFieldAccessTransform implements ILogicalExpressionReferenceTransformWithCondition {
     private final LogicalVariable metaVar;
 
@@ -291,9 +311,8 @@ class MetaKeyToFieldAccessTransform implements ILogicalExpressionReferenceTransf
             return false;
         }
         SourceLocation sourceLoc = expr.getSourceLocation();
-        // Get arguments
-        // first argument : Resource key
-        // second argument: field
+        // get arguments. First argument : Resource key, second argument: field
+        // TODO: how come arg 1 (the data source) is not checked?
         List<Mutable<ILogicalExpression>> args = funcExpr.getArguments();
         ConstantExpression fieldNameExpression = (ConstantExpression) args.get(1).getValue();
         AsterixConstantValue fieldNameValue = (AsterixConstantValue) fieldNameExpression.getValue();
@@ -325,6 +344,21 @@ class MetaKeyToFieldAccessTransform implements ILogicalExpressionReferenceTransf
     }
 }
 
+/**
+ * <pre>
+ * This class replaces meta-key() references with their corresponding logical variables. It maintains a list of
+ * meta-key() references together with their logical variables (the logical variables being the primary key variables
+ * of the data source). For example:
+ * primary key variable (i.e. keyVars): $$1, $$2
+ * meta-key() references (i.e. metaKeyAccessExpressions): meta-key($$ds, "id1"), meta-key($$ds, "id2")
+ *
+ * Any reference to meta-key($$ds, "id1") will be rewritten as $$1.
+ *
+ * meta-key($$ds, "id1") means "access the field id1 of the meta record of data source ds which is also a primary key".
+ *
+ * "id1" and "id2" are the primary keys of the data source "ds". They are fields of the meta record (not $$ds record).
+ * </pre>
+ */
 class MetaKeyExpressionReferenceTransform implements ILogicalExpressionReferenceTransformWithCondition {
     private final List<LogicalVariable> keyVars;
     private final List<ScalarFunctionCallExpression> metaKeyAccessExpressions;
diff --git a/asterixdb/asterix-app/data/csv/people2.csv b/asterixdb/asterix-app/data/csv/people2.csv
new file mode 100644
index 0000000..8488a15
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/people2.csv
@@ -0,0 +1,14 @@
+rec1_US,1,2007-02-02,"{""id"":1, ""name"":""John Mad"", ""age"":29, ""hobby"":""reading""}"
+rec2_US,1,2008-02-04,"{""id"":2, ""name"":""Scott Scott"", ""age"":30, ""hobby"":""hiking""}"
+rec3_UK,2,2018-02-02,"{""id"":3, ""name"":""Dan David"", ""age"":40, ""hobby"":""bowling""}"
+rec4_US,1,2006-01-04,"{""id"":4, ""name"":""Robert Moore"", ""age"":32, ""hobby"":""reading""}"
+rec5_UK,2,2018-01-04,"{""id"":5, ""name"":""Sandy Donald"", ""age"":35, ""hobby"":""soccer""}"
+rec6_US,2,2018-06-05,"{""id"":6, ""name"":""Joe Dana"", ""age"":24, ""hobby"":""tennis""}"
+rec7_US,1,2018-06-07,"{""id"":7, ""name"":""Watson Jordon"", ""age"":28, ""hobby"":""basketball""}"
+rec8_UK,2,2018-06-05,"{""id"":8, ""name"":""Mat Steve"", ""age"":45, ""hobby"":""tennis""}"
+rec9_US,3,2018-06-09,"{""id"":9, ""name"":""Sandra Pec"", ""age"":36, ""hobby"":""hiking""}"
+rec10_US,3,2017-06-09,"{""id"":10, ""name"":""Sandra Lan"", ""age"":36, ""hobby"":""football""}"
+rec11_UK,3,2018-09-09,"{""id"":11, ""name"":""Trever Jones"", ""age"":36, ""hobby"":""hiking""}"
+rec12_UK,3,2016-09-01,"{""id"":12, ""name"":""Ruth Pec"", ""age"":36, ""hobby"":""swimming""}"
+rec13_US,2,2014-09-01,"{""id"":13, ""name"":""Sally Bush"", ""age"":29, ""hobby"":""reading""}"
+rec14_US,1,2010-09-01,"{""id"":14, ""name"":""Sarah Tran"", ""age"":20, ""hobby"":""reading""}"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.1.ddl.sqlpp
new file mode 100644
index 0000000..51b9ffc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.1.ddl.sqlpp
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description  : Create and connect a feed with a filter on meta() fields. Specify the data source for the meta() for
+ *                some datasets and leave it out for others.
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DataType as open {
+id:int,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:string,
+class_type:int,
+updated_date:string
+};
+
+create dataset US_DS1(DataType) with meta(MetaType) primary key meta().id;
+create dataset UK_DS1(DataType) with meta(MetaType) primary key meta().id;
+create dataset CLASS1_DS1(DataType) with meta(MetaType) primary key meta().id;
+create dataset CLASS2_DS1(DataType) with meta(MetaType) primary key meta().id;
+create dataset CLASS3_DS1(DataType) with meta(MetaType) primary key meta().id;
+create dataset REC_B2016_1(DataType) with meta(MetaType) primary key meta().id;
+create dataset REC_A2016_1(DataType) with meta(MetaType) primary key meta().id;
+
+create dataset US_DS2(DataType) with meta(MetaType) primary key meta().id;
+create dataset UK_DS2(DataType) with meta(MetaType) primary key meta().id;
+create dataset CLASS1_DS2(DataType) with meta(MetaType) primary key meta().id;
+create dataset CLASS2_DS2(DataType) with meta(MetaType) primary key meta().id;
+create dataset CLASS3_DS2(DataType) with meta(MetaType) primary key meta().id;
+create dataset REC_B2016_2(DataType) with meta(MetaType) primary key meta().id;
+create dataset REC_A2016_2(DataType) with meta(MetaType) primary key meta().id;
+
+create feed ds_stream with {
+ "adapter-name" : "localfs",
+ "reader" : "localfs",
+ "parser" : "record-with-metadata",
+ "type-name" : "DataType",
+ "meta-type-name" : "MetaType",
+ "path" : "asterix_nc1://data/csv/people2.csv",
+ "format" : "csv",
+ "delimiter" : ",",
+ "record-format" : "adm",
+ "record-index" : "3",
+ "key-indexes" : "0",
+ "key-indicators" : "1",
+ "header" : "false",
+ "change-feed" : "true"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.10.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.10.update.sqlpp
new file mode 100644
index 0000000..c921631
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.10.update.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+set `wait-for-completion-feed` "true";
+connect feed ds_stream to dataset US_DS2 where meta(ds_stream).id like "%US%";
+connect feed ds_stream to dataset UK_DS2 where meta(ds_stream).id like "%UK%";
+connect feed ds_stream to dataset CLASS1_DS2 where meta(ds_stream).class_type = 1;
+connect feed ds_stream to dataset CLASS2_DS2 where meta(ds_stream).class_type = 2;
+connect feed ds_stream to dataset CLASS3_DS2 where meta(ds_stream).class_type = 3;
+connect feed ds_stream to dataset REC_B2016_2 where get_year(date(meta(ds_stream).updated_date)) < 2016;
+connect feed ds_stream to dataset REC_A2016_2 where get_year(date(meta(ds_stream).updated_date)) >= 2016;
+start feed ds_stream;
+
+disconnect feed ds_stream from dataset US_DS2;
+disconnect feed ds_stream from dataset UK_DS2;
+disconnect feed ds_stream from dataset CLASS1_DS2;
+disconnect feed ds_stream from dataset CLASS2_DS2;
+disconnect feed ds_stream from dataset CLASS3_DS2;
+disconnect feed ds_stream from dataset REC_B2016_2;
+disconnect feed ds_stream from dataset REC_A2016_2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.11.query.sqlpp
new file mode 100644
index 0000000..40e6447
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.11.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 value {"rec": v, "meta": meta()} from US_DS2 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.12.query.sqlpp
new file mode 100644
index 0000000..620687b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.12.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 value {"rec": v, "meta": meta()} from UK_DS2 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.13.query.sqlpp
new file mode 100644
index 0000000..63302fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.13.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 value {"rec": v, "meta": meta()} from CLASS1_DS2 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.14.query.sqlpp
new file mode 100644
index 0000000..d9e8108
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.14.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 value {"rec": v, "meta": meta()} from CLASS2_DS2 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.15.query.sqlpp
new file mode 100644
index 0000000..3afc23a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.15.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 value {"rec": v, "meta": meta()} from CLASS3_DS2 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.16.query.sqlpp
new file mode 100644
index 0000000..72b902a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.16.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 value {"rec": v, "meta": meta()} from REC_B2016_2 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.17.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.17.query.sqlpp
new file mode 100644
index 0000000..a80e223
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.17.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 value {"rec": v, "meta": meta()} from REC_A2016_2 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.18.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.18.ddl.sqlpp
new file mode 100644
index 0000000..f12a2b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.18.ddl.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.
+ */
+
+drop dataverse test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.2.update.sqlpp
new file mode 100644
index 0000000..11f81a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.2.update.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+set `wait-for-completion-feed` "true";
+connect feed ds_stream to dataset US_DS1 where meta().id like "%US%";
+connect feed ds_stream to dataset UK_DS1 where meta().id like "%UK%";
+connect feed ds_stream to dataset CLASS1_DS1 where meta().class_type = 1;
+connect feed ds_stream to dataset CLASS2_DS1 where meta().class_type = 2;
+connect feed ds_stream to dataset CLASS3_DS1 where meta().class_type = 3;
+connect feed ds_stream to dataset REC_B2016_1 where get_year(date(meta().updated_date)) < 2016;
+connect feed ds_stream to dataset REC_A2016_1 where get_year(date(meta().updated_date)) >= 2016;
+start feed ds_stream;
+
+disconnect feed ds_stream from dataset US_DS1;
+disconnect feed ds_stream from dataset UK_DS1;
+disconnect feed ds_stream from dataset CLASS1_DS1;
+disconnect feed ds_stream from dataset CLASS2_DS1;
+disconnect feed ds_stream from dataset CLASS3_DS1;
+disconnect feed ds_stream from dataset REC_B2016_1;
+disconnect feed ds_stream from dataset REC_A2016_1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.3.query.sqlpp
new file mode 100644
index 0000000..a4250ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.3.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 value {"rec": v, "meta": meta(v)} from US_DS1 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.4.query.sqlpp
new file mode 100644
index 0000000..e653128
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.4.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 value {"rec": v, "meta": meta()} from UK_DS1 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.5.query.sqlpp
new file mode 100644
index 0000000..a32e706
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.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 value {"rec": v, "meta": meta()} from CLASS1_DS1 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.6.query.sqlpp
new file mode 100644
index 0000000..b505eb6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.6.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value {"rec": v, "meta": meta()} from CLASS2_DS1 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.7.query.sqlpp
new file mode 100644
index 0000000..35cd2c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.7.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 value {"rec": v, "meta": meta()} from CLASS3_DS1 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.8.query.sqlpp
new file mode 100644
index 0000000..476267b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.8.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 value {"rec": v, "meta": meta()} from REC_B2016_1 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.9.query.sqlpp
new file mode 100644
index 0000000..6923b60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.9.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 value {"rec": v, "meta": meta()} from REC_A2016_1 v order by v.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.11.adm
new file mode 100644
index 0000000..cf5fe4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.11.adm
@@ -0,0 +1,9 @@
+{ "rec": { "id": 1, "name": "John Mad", "age": 29, "hobby": "reading" }, "meta": { "id": "rec1_US", "class_type": 1, "updated_date": "2007-02-02" } }
+{ "rec": { "id": 2, "name": "Scott Scott", "age": 30, "hobby": "hiking" }, "meta": { "id": "rec2_US", "class_type": 1, "updated_date": "2008-02-04" } }
+{ "rec": { "id": 4, "name": "Robert Moore", "age": 32, "hobby": "reading" }, "meta": { "id": "rec4_US", "class_type": 1, "updated_date": "2006-01-04" } }
+{ "rec": { "id": 6, "name": "Joe Dana", "age": 24, "hobby": "tennis" }, "meta": { "id": "rec6_US", "class_type": 2, "updated_date": "2018-06-05" } }
+{ "rec": { "id": 7, "name": "Watson Jordon", "age": 28, "hobby": "basketball" }, "meta": { "id": "rec7_US", "class_type": 1, "updated_date": "2018-06-07" } }
+{ "rec": { "id": 9, "name": "Sandra Pec", "age": 36, "hobby": "hiking" }, "meta": { "id": "rec9_US", "class_type": 3, "updated_date": "2018-06-09" } }
+{ "rec": { "id": 10, "name": "Sandra Lan", "age": 36, "hobby": "football" }, "meta": { "id": "rec10_US", "class_type": 3, "updated_date": "2017-06-09" } }
+{ "rec": { "id": 13, "name": "Sally Bush", "age": 29, "hobby": "reading" }, "meta": { "id": "rec13_US", "class_type": 2, "updated_date": "2014-09-01" } }
+{ "rec": { "id": 14, "name": "Sarah Tran", "age": 20, "hobby": "reading" }, "meta": { "id": "rec14_US", "class_type": 1, "updated_date": "2010-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.12.adm
new file mode 100644
index 0000000..9fb07fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.12.adm
@@ -0,0 +1,5 @@
+{ "rec": { "id": 3, "name": "Dan David", "age": 40, "hobby": "bowling" }, "meta": { "id": "rec3_UK", "class_type": 2, "updated_date": "2018-02-02" } }
+{ "rec": { "id": 5, "name": "Sandy Donald", "age": 35, "hobby": "soccer" }, "meta": { "id": "rec5_UK", "class_type": 2, "updated_date": "2018-01-04" } }
+{ "rec": { "id": 8, "name": "Mat Steve", "age": 45, "hobby": "tennis" }, "meta": { "id": "rec8_UK", "class_type": 2, "updated_date": "2018-06-05" } }
+{ "rec": { "id": 11, "name": "Trever Jones", "age": 36, "hobby": "hiking" }, "meta": { "id": "rec11_UK", "class_type": 3, "updated_date": "2018-09-09" } }
+{ "rec": { "id": 12, "name": "Ruth Pec", "age": 36, "hobby": "swimming" }, "meta": { "id": "rec12_UK", "class_type": 3, "updated_date": "2016-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.13.adm
new file mode 100644
index 0000000..2cd20f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.13.adm
@@ -0,0 +1,5 @@
+{ "rec": { "id": 1, "name": "John Mad", "age": 29, "hobby": "reading" }, "meta": { "id": "rec1_US", "class_type": 1, "updated_date": "2007-02-02" } }
+{ "rec": { "id": 2, "name": "Scott Scott", "age": 30, "hobby": "hiking" }, "meta": { "id": "rec2_US", "class_type": 1, "updated_date": "2008-02-04" } }
+{ "rec": { "id": 4, "name": "Robert Moore", "age": 32, "hobby": "reading" }, "meta": { "id": "rec4_US", "class_type": 1, "updated_date": "2006-01-04" } }
+{ "rec": { "id": 7, "name": "Watson Jordon", "age": 28, "hobby": "basketball" }, "meta": { "id": "rec7_US", "class_type": 1, "updated_date": "2018-06-07" } }
+{ "rec": { "id": 14, "name": "Sarah Tran", "age": 20, "hobby": "reading" }, "meta": { "id": "rec14_US", "class_type": 1, "updated_date": "2010-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.14.adm
new file mode 100644
index 0000000..4a0957e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.14.adm
@@ -0,0 +1,5 @@
+{ "rec": { "id": 3, "name": "Dan David", "age": 40, "hobby": "bowling" }, "meta": { "id": "rec3_UK", "class_type": 2, "updated_date": "2018-02-02" } }
+{ "rec": { "id": 5, "name": "Sandy Donald", "age": 35, "hobby": "soccer" }, "meta": { "id": "rec5_UK", "class_type": 2, "updated_date": "2018-01-04" } }
+{ "rec": { "id": 6, "name": "Joe Dana", "age": 24, "hobby": "tennis" }, "meta": { "id": "rec6_US", "class_type": 2, "updated_date": "2018-06-05" } }
+{ "rec": { "id": 8, "name": "Mat Steve", "age": 45, "hobby": "tennis" }, "meta": { "id": "rec8_UK", "class_type": 2, "updated_date": "2018-06-05" } }
+{ "rec": { "id": 13, "name": "Sally Bush", "age": 29, "hobby": "reading" }, "meta": { "id": "rec13_US", "class_type": 2, "updated_date": "2014-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.15.adm
new file mode 100644
index 0000000..5f119c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.15.adm
@@ -0,0 +1,4 @@
+{ "rec": { "id": 9, "name": "Sandra Pec", "age": 36, "hobby": "hiking" }, "meta": { "id": "rec9_US", "class_type": 3, "updated_date": "2018-06-09" } }
+{ "rec": { "id": 10, "name": "Sandra Lan", "age": 36, "hobby": "football" }, "meta": { "id": "rec10_US", "class_type": 3, "updated_date": "2017-06-09" } }
+{ "rec": { "id": 11, "name": "Trever Jones", "age": 36, "hobby": "hiking" }, "meta": { "id": "rec11_UK", "class_type": 3, "updated_date": "2018-09-09" } }
+{ "rec": { "id": 12, "name": "Ruth Pec", "age": 36, "hobby": "swimming" }, "meta": { "id": "rec12_UK", "class_type": 3, "updated_date": "2016-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.16.adm
new file mode 100644
index 0000000..303b8b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.16.adm
@@ -0,0 +1,5 @@
+{ "rec": { "id": 1, "name": "John Mad", "age": 29, "hobby": "reading" }, "meta": { "id": "rec1_US", "class_type": 1, "updated_date": "2007-02-02" } }
+{ "rec": { "id": 2, "name": "Scott Scott", "age": 30, "hobby": "hiking" }, "meta": { "id": "rec2_US", "class_type": 1, "updated_date": "2008-02-04" } }
+{ "rec": { "id": 4, "name": "Robert Moore", "age": 32, "hobby": "reading" }, "meta": { "id": "rec4_US", "class_type": 1, "updated_date": "2006-01-04" } }
+{ "rec": { "id": 13, "name": "Sally Bush", "age": 29, "hobby": "reading" }, "meta": { "id": "rec13_US", "class_type": 2, "updated_date": "2014-09-01" } }
+{ "rec": { "id": 14, "name": "Sarah Tran", "age": 20, "hobby": "reading" }, "meta": { "id": "rec14_US", "class_type": 1, "updated_date": "2010-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.17.adm
new file mode 100644
index 0000000..0a93b0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.17.adm
@@ -0,0 +1,9 @@
+{ "rec": { "id": 3, "name": "Dan David", "age": 40, "hobby": "bowling" }, "meta": { "id": "rec3_UK", "class_type": 2, "updated_date": "2018-02-02" } }
+{ "rec": { "id": 5, "name": "Sandy Donald", "age": 35, "hobby": "soccer" }, "meta": { "id": "rec5_UK", "class_type": 2, "updated_date": "2018-01-04" } }
+{ "rec": { "id": 6, "name": "Joe Dana", "age": 24, "hobby": "tennis" }, "meta": { "id": "rec6_US", "class_type": 2, "updated_date": "2018-06-05" } }
+{ "rec": { "id": 7, "name": "Watson Jordon", "age": 28, "hobby": "basketball" }, "meta": { "id": "rec7_US", "class_type": 1, "updated_date": "2018-06-07" } }
+{ "rec": { "id": 8, "name": "Mat Steve", "age": 45, "hobby": "tennis" }, "meta": { "id": "rec8_UK", "class_type": 2, "updated_date": "2018-06-05" } }
+{ "rec": { "id": 9, "name": "Sandra Pec", "age": 36, "hobby": "hiking" }, "meta": { "id": "rec9_US", "class_type": 3, "updated_date": "2018-06-09" } }
+{ "rec": { "id": 10, "name": "Sandra Lan", "age": 36, "hobby": "football" }, "meta": { "id": "rec10_US", "class_type": 3, "updated_date": "2017-06-09" } }
+{ "rec": { "id": 11, "name": "Trever Jones", "age": 36, "hobby": "hiking" }, "meta": { "id": "rec11_UK", "class_type": 3, "updated_date": "2018-09-09" } }
+{ "rec": { "id": 12, "name": "Ruth Pec", "age": 36, "hobby": "swimming" }, "meta": { "id": "rec12_UK", "class_type": 3, "updated_date": "2016-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.3.adm
new file mode 100644
index 0000000..cf5fe4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.3.adm
@@ -0,0 +1,9 @@
+{ "rec": { "id": 1, "name": "John Mad", "age": 29, "hobby": "reading" }, "meta": { "id": "rec1_US", "class_type": 1, "updated_date": "2007-02-02" } }
+{ "rec": { "id": 2, "name": "Scott Scott", "age": 30, "hobby": "hiking" }, "meta": { "id": "rec2_US", "class_type": 1, "updated_date": "2008-02-04" } }
+{ "rec": { "id": 4, "name": "Robert Moore", "age": 32, "hobby": "reading" }, "meta": { "id": "rec4_US", "class_type": 1, "updated_date": "2006-01-04" } }
+{ "rec": { "id": 6, "name": "Joe Dana", "age": 24, "hobby": "tennis" }, "meta": { "id": "rec6_US", "class_type": 2, "updated_date": "2018-06-05" } }
+{ "rec": { "id": 7, "name": "Watson Jordon", "age": 28, "hobby": "basketball" }, "meta": { "id": "rec7_US", "class_type": 1, "updated_date": "2018-06-07" } }
+{ "rec": { "id": 9, "name": "Sandra Pec", "age": 36, "hobby": "hiking" }, "meta": { "id": "rec9_US", "class_type": 3, "updated_date": "2018-06-09" } }
+{ "rec": { "id": 10, "name": "Sandra Lan", "age": 36, "hobby": "football" }, "meta": { "id": "rec10_US", "class_type": 3, "updated_date": "2017-06-09" } }
+{ "rec": { "id": 13, "name": "Sally Bush", "age": 29, "hobby": "reading" }, "meta": { "id": "rec13_US", "class_type": 2, "updated_date": "2014-09-01" } }
+{ "rec": { "id": 14, "name": "Sarah Tran", "age": 20, "hobby": "reading" }, "meta": { "id": "rec14_US", "class_type": 1, "updated_date": "2010-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.4.adm
new file mode 100644
index 0000000..9fb07fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.4.adm
@@ -0,0 +1,5 @@
+{ "rec": { "id": 3, "name": "Dan David", "age": 40, "hobby": "bowling" }, "meta": { "id": "rec3_UK", "class_type": 2, "updated_date": "2018-02-02" } }
+{ "rec": { "id": 5, "name": "Sandy Donald", "age": 35, "hobby": "soccer" }, "meta": { "id": "rec5_UK", "class_type": 2, "updated_date": "2018-01-04" } }
+{ "rec": { "id": 8, "name": "Mat Steve", "age": 45, "hobby": "tennis" }, "meta": { "id": "rec8_UK", "class_type": 2, "updated_date": "2018-06-05" } }
+{ "rec": { "id": 11, "name": "Trever Jones", "age": 36, "hobby": "hiking" }, "meta": { "id": "rec11_UK", "class_type": 3, "updated_date": "2018-09-09" } }
+{ "rec": { "id": 12, "name": "Ruth Pec", "age": 36, "hobby": "swimming" }, "meta": { "id": "rec12_UK", "class_type": 3, "updated_date": "2016-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.5.adm
new file mode 100644
index 0000000..2cd20f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.5.adm
@@ -0,0 +1,5 @@
+{ "rec": { "id": 1, "name": "John Mad", "age": 29, "hobby": "reading" }, "meta": { "id": "rec1_US", "class_type": 1, "updated_date": "2007-02-02" } }
+{ "rec": { "id": 2, "name": "Scott Scott", "age": 30, "hobby": "hiking" }, "meta": { "id": "rec2_US", "class_type": 1, "updated_date": "2008-02-04" } }
+{ "rec": { "id": 4, "name": "Robert Moore", "age": 32, "hobby": "reading" }, "meta": { "id": "rec4_US", "class_type": 1, "updated_date": "2006-01-04" } }
+{ "rec": { "id": 7, "name": "Watson Jordon", "age": 28, "hobby": "basketball" }, "meta": { "id": "rec7_US", "class_type": 1, "updated_date": "2018-06-07" } }
+{ "rec": { "id": 14, "name": "Sarah Tran", "age": 20, "hobby": "reading" }, "meta": { "id": "rec14_US", "class_type": 1, "updated_date": "2010-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.6.adm
new file mode 100644
index 0000000..4a0957e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.6.adm
@@ -0,0 +1,5 @@
+{ "rec": { "id": 3, "name": "Dan David", "age": 40, "hobby": "bowling" }, "meta": { "id": "rec3_UK", "class_type": 2, "updated_date": "2018-02-02" } }
+{ "rec": { "id": 5, "name": "Sandy Donald", "age": 35, "hobby": "soccer" }, "meta": { "id": "rec5_UK", "class_type": 2, "updated_date": "2018-01-04" } }
+{ "rec": { "id": 6, "name": "Joe Dana", "age": 24, "hobby": "tennis" }, "meta": { "id": "rec6_US", "class_type": 2, "updated_date": "2018-06-05" } }
+{ "rec": { "id": 8, "name": "Mat Steve", "age": 45, "hobby": "tennis" }, "meta": { "id": "rec8_UK", "class_type": 2, "updated_date": "2018-06-05" } }
+{ "rec": { "id": 13, "name": "Sally Bush", "age": 29, "hobby": "reading" }, "meta": { "id": "rec13_US", "class_type": 2, "updated_date": "2014-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.7.adm
new file mode 100644
index 0000000..5f119c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.7.adm
@@ -0,0 +1,4 @@
+{ "rec": { "id": 9, "name": "Sandra Pec", "age": 36, "hobby": "hiking" }, "meta": { "id": "rec9_US", "class_type": 3, "updated_date": "2018-06-09" } }
+{ "rec": { "id": 10, "name": "Sandra Lan", "age": 36, "hobby": "football" }, "meta": { "id": "rec10_US", "class_type": 3, "updated_date": "2017-06-09" } }
+{ "rec": { "id": 11, "name": "Trever Jones", "age": 36, "hobby": "hiking" }, "meta": { "id": "rec11_UK", "class_type": 3, "updated_date": "2018-09-09" } }
+{ "rec": { "id": 12, "name": "Ruth Pec", "age": 36, "hobby": "swimming" }, "meta": { "id": "rec12_UK", "class_type": 3, "updated_date": "2016-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.8.adm
new file mode 100644
index 0000000..303b8b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.8.adm
@@ -0,0 +1,5 @@
+{ "rec": { "id": 1, "name": "John Mad", "age": 29, "hobby": "reading" }, "meta": { "id": "rec1_US", "class_type": 1, "updated_date": "2007-02-02" } }
+{ "rec": { "id": 2, "name": "Scott Scott", "age": 30, "hobby": "hiking" }, "meta": { "id": "rec2_US", "class_type": 1, "updated_date": "2008-02-04" } }
+{ "rec": { "id": 4, "name": "Robert Moore", "age": 32, "hobby": "reading" }, "meta": { "id": "rec4_US", "class_type": 1, "updated_date": "2006-01-04" } }
+{ "rec": { "id": 13, "name": "Sally Bush", "age": 29, "hobby": "reading" }, "meta": { "id": "rec13_US", "class_type": 2, "updated_date": "2014-09-01" } }
+{ "rec": { "id": 14, "name": "Sarah Tran", "age": 20, "hobby": "reading" }, "meta": { "id": "rec14_US", "class_type": 1, "updated_date": "2010-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.9.adm
new file mode 100644
index 0000000..0a93b0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/change-feed-with-filter-on-meta/change-feed-with-filter-on-meta.9.adm
@@ -0,0 +1,9 @@
+{ "rec": { "id": 3, "name": "Dan David", "age": 40, "hobby": "bowling" }, "meta": { "id": "rec3_UK", "class_type": 2, "updated_date": "2018-02-02" } }
+{ "rec": { "id": 5, "name": "Sandy Donald", "age": 35, "hobby": "soccer" }, "meta": { "id": "rec5_UK", "class_type": 2, "updated_date": "2018-01-04" } }
+{ "rec": { "id": 6, "name": "Joe Dana", "age": 24, "hobby": "tennis" }, "meta": { "id": "rec6_US", "class_type": 2, "updated_date": "2018-06-05" } }
+{ "rec": { "id": 7, "name": "Watson Jordon", "age": 28, "hobby": "basketball" }, "meta": { "id": "rec7_US", "class_type": 1, "updated_date": "2018-06-07" } }
+{ "rec": { "id": 8, "name": "Mat Steve", "age": 45, "hobby": "tennis" }, "meta": { "id": "rec8_UK", "class_type": 2, "updated_date": "2018-06-05" } }
+{ "rec": { "id": 9, "name": "Sandra Pec", "age": 36, "hobby": "hiking" }, "meta": { "id": "rec9_US", "class_type": 3, "updated_date": "2018-06-09" } }
+{ "rec": { "id": 10, "name": "Sandra Lan", "age": 36, "hobby": "football" }, "meta": { "id": "rec10_US", "class_type": 3, "updated_date": "2017-06-09" } }
+{ "rec": { "id": 11, "name": "Trever Jones", "age": 36, "hobby": "hiking" }, "meta": { "id": "rec11_UK", "class_type": 3, "updated_date": "2018-09-09" } }
+{ "rec": { "id": 12, "name": "Ruth Pec", "age": 36, "hobby": "swimming" }, "meta": { "id": "rec12_UK", "class_type": 3, "updated_date": "2016-09-01" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 18c5f88..e49b36d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -9056,6 +9056,11 @@
         <expected-error>Function fundv.test_func0@1 is being used. It cannot be dropped</expected-error>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="change-feed-with-filter-on-meta">
+        <output-dir compare="Text">change-feed-with-filter-on-meta</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="meta">
     <test-case FilePath="meta">