[ASTERIXDB-1254][EXT] Add Apache Parquet as external format

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

Details:
- Add the support for Apache Parquet format as an external one
- Add new compiler property compiler.expression.pushdowns
  which allows us to pushdown field-accesses down to data-scan.
  Currently, this is only supported for Parquet input-format.
- Allow single-NC configuration to utilize Hadoop formats without
  the need for HDFS.

Changes:
- Add IProjectionInfo to hold the projected field names
  information for data-scan
- Add IFieldValue to extend Parquet Converter to get values' field names.
- Add NoOpDataParserFactory and NoOpDataParser.

Change-Id: I00f3625b332c298ea93b5cab44cb648c89075079
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/6904
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dmitry.lychagin@couchbase.com>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 57ae375..4a2b629 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -68,6 +68,7 @@
 import org.apache.asterix.optimizer.rules.PushAggFuncIntoStandaloneAggregateRule;
 import org.apache.asterix.optimizer.rules.PushAggregateIntoNestedSubplanRule;
 import org.apache.asterix.optimizer.rules.PushFieldAccessRule;
+import org.apache.asterix.optimizer.rules.PushFieldAccessToExternalDataScanRule;
 import org.apache.asterix.optimizer.rules.PushGroupByThroughProduct;
 import org.apache.asterix.optimizer.rules.PushLimitIntoOrderByRule;
 import org.apache.asterix.optimizer.rules.PushLimitIntoPrimarySearchRule;
@@ -387,6 +388,7 @@
         // remove assigns that could become unused after PushLimitIntoPrimarySearchRule
         physicalRewritesTopLevel.add(new RemoveUnusedAssignAndAggregateRule());
         physicalRewritesTopLevel.add(new IntroduceProjectsRule());
+        physicalRewritesTopLevel.add(new PushFieldAccessToExternalDataScanRule());
         physicalRewritesTopLevel.add(new SetAsterixPhysicalOperatorsRule());
         physicalRewritesTopLevel.add(new IntroduceRapidFrameFlushProjectAssignRule());
         physicalRewritesTopLevel.add(new SetExecutionModeRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessToExternalDataScanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessToExternalDataScanRule.java
new file mode 100644
index 0000000..f25e058
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushFieldAccessToExternalDataScanRule.java
@@ -0,0 +1,248 @@
+/*
+ * 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.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.metadata.declared.DatasetDataSource;
+import org.apache.asterix.metadata.declared.ExternalDataProjectionInfo;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Pushes field-access expression to the external dataset scan to minimize the size of the record.
+ * This rule currently does not remove the field access expression in ASSIGN and SCAN operators. Instead,
+ * it adds the requested field names to external dataset details to produce records that only contain the requested
+ * fields. Thus, no changes would occur in the plan's structure after firing this rule.
+ * Example:
+ * Before plan:
+ * ...
+ * select (and(gt($$00, 20), gt($$r.getField("salary"), 70000)))
+ * ...
+ * assign [$$00] <- [$$r.getField("personalInfo").getField("age")]
+ * ...
+ * data-scan []<-[$$r] <- ParquetDataverse.ParquetDataset
+ * <p>
+ * After plan:
+ * ...
+ * select (and(gt($$00, 20), gt($$r.getField("salary"), 70000)))
+ * ...
+ * assign [$$00] <- [$$r.getField("personalInfo").getField("age")]
+ * ...
+ * data-scan []<-[$$r] <- ParquetDataverse.ParquetDataset project (personalInfo.age, salary)
+ * <p>
+ * The resulting record $$r will be {"personalInfo":{"age": *AGE*}, "salary": *SALARY*}
+ * and other fields will not be included in $$r.
+ */
+public class PushFieldAccessToExternalDataScanRule implements IAlgebraicRewriteRule {
+    //Datasets payload variables
+    private final List<LogicalVariable> recordVariables = new ArrayList<>();
+    //Dataset scan operators' projection info
+    private final List<ExternalDataProjectionInfo> projectionInfos = new ArrayList<>();
+    //Final result live variables
+    private final Set<LogicalVariable> projectedVariables = new HashSet<>();
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        final ILogicalOperator currentOp = opRef.getValue();
+        final LogicalOperatorTag currentOpTag = currentOp.getOperatorTag();
+        if (!context.getPhysicalOptimizationConfig().isExternalFieldPushdown()) {
+            return false;
+        }
+        if (currentOpTag == LogicalOperatorTag.PROJECT) {
+            ProjectOperator projectOp = (ProjectOperator) currentOp;
+            projectedVariables.addAll(projectOp.getVariables());
+            return false;
+        }
+
+        if (currentOpTag != LogicalOperatorTag.DATASOURCESCAN) {
+            return false;
+        }
+
+        return setDatasetProperties(currentOp, (MetadataProvider) context.getMetadataProvider());
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        final ILogicalOperator op = opRef.getValue();
+        if (!context.getPhysicalOptimizationConfig().isExternalFieldPushdown()
+                || context.checkIfInDontApplySet(this, op) || projectionInfos.isEmpty()) {
+            return false;
+        }
+
+        if (op.getOperatorTag() != LogicalOperatorTag.SELECT && op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+            return false;
+        }
+
+        if (op.getOperatorTag() == LogicalOperatorTag.SELECT) {
+            final SelectOperator selectOp = (SelectOperator) op;
+            pushFieldAccessExpression(selectOp.getCondition(), context);
+        } else {
+            final AssignOperator assignOp = (AssignOperator) op;
+            pushFieldAccessExpression(assignOp.getExpressions(), context);
+        }
+
+        //Add to do not apply to avoid pushing the same expression twice when the plan contains REPLICATE
+        context.addToDontApplySet(this, op);
+
+        return false;
+    }
+
+    private void pushFieldAccessExpression(List<Mutable<ILogicalExpression>> exprList, IOptimizationContext context)
+            throws AlgebricksException {
+
+        for (Mutable<ILogicalExpression> exprRef : exprList) {
+            pushFieldAccessExpression(exprRef, context);
+        }
+    }
+
+    private void pushFieldAccessExpression(Mutable<ILogicalExpression> exprRef, IOptimizationContext context)
+            throws AlgebricksException {
+        final ILogicalExpression expr = exprRef.getValue();
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return;
+        }
+
+        final AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+
+        //Only field access expressions are allowed
+        if (!isFieldAccessByName(funcExpr)) {
+            pushFieldAccessExpression(funcExpr.getArguments(), context);
+            return;
+        }
+
+        //Get root expression input variable in case it is nested field access
+        final LogicalVariable funcRootInputVar = getRootExpressionInputVariable(funcExpr);
+        if (funcRootInputVar != null) {
+            final int recordVarIndex = recordVariables.indexOf(funcRootInputVar);
+            //Is funcRootInputVar originated from a data-scan operator?
+            if (recordVarIndex >= 0) {
+                final List<List<String>> projectedFieldNames = projectionInfos.get(recordVarIndex).getProjectionInfo();
+                final List<String> fieldNames = new ArrayList<>();
+                //Add fieldAccessExpr to field names list
+                buildFieldNames(funcExpr, fieldNames);
+                if (!fieldNames.isEmpty()) {
+                    projectedFieldNames.add(fieldNames);
+                }
+            }
+        } else {
+            //Descend to the arguments expressions to see if any can be pushed
+            pushFieldAccessExpression(funcExpr.getArguments(), context);
+        }
+    }
+
+    private boolean setDatasetProperties(ILogicalOperator op, MetadataProvider mp) throws AlgebricksException {
+        final DataSourceScanOperator scan = (DataSourceScanOperator) op;
+        final DataSource dataSource = (DataSource) scan.getDataSource();
+
+        if (dataSource == null) {
+            return false;
+        }
+        final DataverseName dataverse = dataSource.getId().getDataverseName();
+        final String datasetName = dataSource.getId().getDatasourceName();
+        final Dataset dataset = mp.findDataset(dataverse, datasetName);
+
+        //Only external dataset can have pushed down expressions
+        if (dataset == null || dataset.getDatasetType() == DatasetType.INTERNAL
+                || dataset.getDatasetType() == DatasetType.EXTERNAL && !ExternalDataUtils
+                        .supportsPushdown(((ExternalDatasetDetails) dataset.getDatasetDetails()).getProperties())) {
+            return false;
+        }
+
+        boolean changed = false;
+        final DatasetDataSource datasetDataSource = (DatasetDataSource) dataSource;
+        final LogicalVariable recordVar = datasetDataSource.getDataRecordVariable(scan.getVariables());
+        if (!projectedVariables.contains(recordVar) && scan.getProjectionInfo() == null) {
+            //Do not push expressions to data scan if the whole record is needed
+            recordVariables.add(recordVar);
+            ExternalDataProjectionInfo projectionInfo = new ExternalDataProjectionInfo();
+            scan.setProjectionInfo(projectionInfo);
+            projectionInfos.add(projectionInfo);
+            changed = true;
+        }
+        return changed;
+    }
+
+    private static LogicalVariable getRootExpressionInputVariable(AbstractFunctionCallExpression funcExpr) {
+        ILogicalExpression currentExpr = funcExpr.getArguments().get(0).getValue();
+        while (isFieldAccessByName(currentExpr)) {
+            currentExpr = ((AbstractFunctionCallExpression) currentExpr).getArguments().get(0).getValue();
+        }
+
+        if (currentExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+            return ((VariableReferenceExpression) currentExpr).getVariableReference();
+        }
+        return null;
+    }
+
+    private static boolean isFieldAccessByName(ILogicalExpression expression) {
+        return expression.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
+                && BuiltinFunctions.FIELD_ACCESS_BY_NAME
+                        .equals(((AbstractFunctionCallExpression) expression).getFunctionIdentifier());
+    }
+
+    private static void buildFieldNames(ILogicalExpression expr, List<String> fieldNames) throws CompilationException {
+        if (!isFieldAccessByName(expr)) {
+            /*
+             * We only push nested field-access expressions.
+             * This is a sanity check if the previous checks have missed.
+             */
+            return;
+        }
+
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        ILogicalExpression objectExpr = funcExpr.getArguments().get(0).getValue();
+        if (!isPayload(objectExpr)) {
+            buildFieldNames(objectExpr, fieldNames);
+        }
+        fieldNames.add(ConstantExpressionUtil.getStringArgument(funcExpr, 1));
+    }
+
+    private static boolean isPayload(ILogicalExpression expr) {
+        return expr.getExpressionTag() == LogicalExpressionTag.VARIABLE;
+    }
+}
diff --git a/asterixdb/asterix-app/data/hdfs/parquet/dummy_tweet.json b/asterixdb/asterix-app/data/hdfs/parquet/dummy_tweet.json
new file mode 100644
index 0000000..5ca0b05
--- /dev/null
+++ b/asterixdb/asterix-app/data/hdfs/parquet/dummy_tweet.json
@@ -0,0 +1,2 @@
+{  "coordinates": {"coordinates": [1.1],"type": "string"  },  "created_at": "string",  "entities": {"urls": [{  "display_url": "string",  "expanded_url": "string",  "indices": [1],  "url": "string"}],"user_mentions": [{  "id": 1,  "id_str": "string",  "indices": [1],  "name": "string",  "screen_name": "string"}]  },  "favorite_count": 1,  "favorited": true,  "filter_level": "string",  "geo": {"coordinates": [1.1],"type": "string"  },  "id": "0000000",  "id_str": "string",  "in_reply_to_screen_name": "string",  "in_reply_to_status_id": 1,  "in_reply_to_status_id_str": "string",  "in_reply_to_user_id": 1,  "in_reply_to_user_id_str": "string",  "is_quote_status": true,  "lang": "string",  "place": {"bounding_box": {  "coordinates": [[[1.1]]],  "type": "string"},"country": "string","country_code": "string","full_name": "string","id": "string","name": "string","place_type": "string","url": "string"  },  "possibly_sensitive": true,  "quoted_status": {"created_at": "string","entities": {"user_mentions": [{  "id": 1,  "id_str": "string",  "indices": [1],  "name": "string",  "screen_name": "string"}]},"favorite_count": 1,"favorited": true,"filter_level": "string","id": 1,"id_str": "string","in_reply_to_screen_name": "string","in_reply_to_status_id": 1,"in_reply_to_status_id_str": "string","in_reply_to_user_id": 1,"in_reply_to_user_id_str": "string","is_quote_status": true,"lang": "string","retweet_count": 1,"retweeted": true,"source": "string","text": "string","truncated": true,"user": {  "contributors_enabled": true,  "created_at": "string",  "default_profile": true,  "default_profile_image": true,  "description": "string",  "favourites_count": 1,  "followers_count": 1,  "friends_count": 1,  "geo_enabled": true,  "id": 1,  "id_str": "string",  "is_translator": true,  "lang": "string",  "listed_count": 1,  "name": "string",  "profile_background_color": "string",  "profile_background_image_url": "string",  "profile_background_image_url_https": "string",  "profile_background_tile": true,  "profile_banner_url": "string",  "profile_image_url": "string",  "profile_image_url_https": "string",  "profile_link_color": "string",  "profile_sidebar_border_color": "string",  "profile_sidebar_fill_color": "string",  "profile_text_color": "string",  "profile_use_background_image": true,  "protected": true,  "screen_name": "string",  "statuses_count": 1,  "verified": true}  },  "quoted_status_id": 1,  "quoted_status_id_str": "string",  "retweet_count": 1,  "retweeted": true,  "source": "string",  "text": "string",  "timestamp_ms": "string",  "truncated": true,  "user": {"contributors_enabled": true,"created_at": "string","default_profile": true,"default_profile_image": true,"description": "string","favourites_count": 1,"followers_count": 1,"friends_count": 1,"geo_enabled": true,"id": 1,"id_str": "string","is_translator": true,"lang": "string","listed_count": 1,"location": "string","name": "string","profile_background_color": "string","profile_background_image_url": "string","profile_background_image_url_https": "string","profile_background_tile": true,"profile_banner_url": "string","profile_image_url": "string","profile_image_url_https": "string","profile_link_color": "string","profile_sidebar_border_color": "string","profile_sidebar_fill_color": "string","profile_text_color": "string","profile_use_background_image": true,"protected": true,"screen_name": "string","statuses_count": 1,"time_zone": "string","url": "string","utc_offset": 1,"verified": true  }}
+{  "coordinates": {"coordinates": [1.1],"type": "string"  },  "created_at": "string",  "favorite_count": 1,  "favorited": true,  "filter_level": "string",  "geo": {"coordinates": [1.1],"type": "string"  },  "id": "11111111111111111111",  "id_str": "string",  "in_reply_to_screen_name": "string",  "in_reply_to_status_id": 1,  "in_reply_to_status_id_str": "string",  "in_reply_to_user_id": 1,  "in_reply_to_user_id_str": "string",  "is_quote_status": true,  "lang": "string",  "place": {"bounding_box": {  "coordinates": [[[1.1]]],  "type": "string"},"country": "string","country_code": "string","full_name": "string","id": "string","name": "string","place_type": "string","url": "string"  },  "possibly_sensitive": true,  "quoted_status": {"created_at": "string","entities": {"user_mentions": [{  "id": 1,  "id_str": "string",  "indices": [1],  "name": "string",  "screen_name": "string"}]},"favorite_count": 1,"favorited": true,"filter_level": "string","id": 1,"id_str": "string","in_reply_to_screen_name": "string","in_reply_to_status_id": 1,"in_reply_to_status_id_str": "string","in_reply_to_user_id": 1,"in_reply_to_user_id_str": "string","is_quote_status": true,"lang": "string","retweet_count": 1,"retweeted": true,"source": "string","text": "string","truncated": true,"user": {  "contributors_enabled": true,  "created_at": "string",  "default_profile": true,  "default_profile_image": true,  "description": "string",  "favourites_count": 1,  "followers_count": 1,  "friends_count": 1,  "geo_enabled": true,  "id": 1,  "id_str": "string",  "is_translator": true,  "lang": "string",  "listed_count": 1,  "name": "string",  "profile_background_color": "string",  "profile_background_image_url": "string",  "profile_background_image_url_https": "string",  "profile_background_tile": true,  "profile_banner_url": "string",  "profile_image_url": "string",  "profile_image_url_https": "string",  "profile_link_color": "string",  "profile_sidebar_border_color": "string",  "profile_sidebar_fill_color": "string",  "profile_text_color": "string",  "profile_use_background_image": true,  "protected": true,  "screen_name": "string",  "statuses_count": 1,  "verified": true}  },  "quoted_status_id": 1,  "quoted_status_id_str": "string",  "retweet_count": 1,  "retweeted": true,  "source": "string",  "text": "string",  "timestamp_ms": "string",  "truncated": true,  "user": {"contributors_enabled": true,"created_at": "string","default_profile": true,"default_profile_image": true,"description": "string","favourites_count": 1,"followers_count": 1,"friends_count": 1,"geo_enabled": true,"id": 1,"id_str": "string","is_translator": true,"lang": "string","listed_count": 1,"location": "string","name": "string","profile_background_color": "string","profile_background_image_url": "string","profile_background_image_url_https": "string","profile_background_tile": true,"profile_banner_url": "string","profile_image_url": "string","profile_image_url_https": "string","profile_link_color": "string","profile_sidebar_border_color": "string","profile_sidebar_fill_color": "string","profile_text_color": "string","profile_use_background_image": true,"protected": true,"screen_name": "string","statuses_count": 1,"time_zone": "string","url": "string","utc_offset": 1,"verified": true  }}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/hdfs/parquet/id_age.json b/asterixdb/asterix-app/data/hdfs/parquet/id_age.json
new file mode 100644
index 0000000..07bfd6b
--- /dev/null
+++ b/asterixdb/asterix-app/data/hdfs/parquet/id_age.json
@@ -0,0 +1,7 @@
+{"id": 8, "age": 10}
+{"id": 9, "age": 20}
+{"id": 10, "age": 30}
+{"id": 11, "age": 40}
+{"id": 12, "age": 50}
+{"id": 13, "age": 60}
+{"id": 14, "age": 70}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/hdfs/parquet/id_name.json b/asterixdb/asterix-app/data/hdfs/parquet/id_name.json
new file mode 100644
index 0000000..22d72d3
--- /dev/null
+++ b/asterixdb/asterix-app/data/hdfs/parquet/id_name.json
@@ -0,0 +1,7 @@
+{ "id": 1, "name": "John" }
+{ "id": 2, "name": "Abel" }
+{ "id": 3, "name": "Sandy" }
+{ "id": 4, "name": "Alex" }
+{ "id": 5, "name": "Mike" }
+{ "id": 6, "name": "Tom" }
+{ "id": 7, "name": "Jerry" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/hdfs/parquet/id_name_comment.json b/asterixdb/asterix-app/data/hdfs/parquet/id_name_comment.json
new file mode 100644
index 0000000..9ba8b2d
--- /dev/null
+++ b/asterixdb/asterix-app/data/hdfs/parquet/id_name_comment.json
@@ -0,0 +1,8 @@
+{"id": 1, "name": "John"}
+{"id": 2, "name": "Abel"}
+{"id": 3, "name": "Sandy"}
+{"id": 4, "name": "Alex"}
+{"id": 5, "name": "Mike"}
+{"id": 6, "name": "Tom"}
+{"id": 7, "name": "Jerry", "comment": "\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا"}
+{"id": 8, "name": "William", "comment": "\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا\uD83D\uDE22\uD83D\uDE22\uD83D\uDC89\uD83D\uDC89 = 𩸽 😢😢💉💉. Coffee ☕‼️😃. حسنا"}
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index 7bf5127..7f8085d 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -864,5 +864,11 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-1.2-api</artifactId>
     </dependency>
+    <!-- Needed for inferring the schema for writing Parquet files -->
+    <dependency>
+      <groupId>org.kitesdk</groupId>
+      <artifactId>kite-data-core</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index a5daab6..7a86f65 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -134,11 +134,12 @@
                     CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY, CompilerProperties.COMPILER_PARALLELISM_KEY,
                     CompilerProperties.COMPILER_SORT_PARALLEL_KEY, CompilerProperties.COMPILER_SORT_SAMPLES_KEY,
                     CompilerProperties.COMPILER_INDEXONLY_KEY, CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
-                    FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, FuzzyUtils.SIM_FUNCTION_PROP_NAME,
-                    FuzzyUtils.SIM_THRESHOLD_PROP_NAME, StartFeedStatement.WAIT_FOR_COMPLETION,
-                    FeedActivityDetails.FEED_POLICY_NAME, FeedActivityDetails.COLLECT_LOCATIONS,
-                    SqlppQueryRewriter.INLINE_WITH_OPTION, SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION,
-                    "hash_merge", "output-record-type", DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION);
+                    CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
+                    FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
+                    StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
+                    FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
+                    SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
+                    DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION);
 
     private final IRewriterFactory rewriterFactory;
     private final IAstPrintVisitorFactory astPrintVisitorFactory;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/HDFSCluster.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/HDFSCluster.java
index c0b4841..f5c51b4 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/HDFSCluster.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/HDFSCluster.java
@@ -19,9 +19,14 @@
 package org.apache.asterix.test.runtime;
 
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
 
 import org.apache.asterix.external.dataset.adapter.GenericAdapter;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData.Record;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -31,6 +36,11 @@
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hyracks.api.util.IoUtil;
+import org.kitesdk.data.spi.JsonUtil;
+import org.kitesdk.data.spi.filesystem.JSONFileReader;
+
+import parquet.avro.AvroParquetWriter;
 
 /**
  * Manages a Mini (local VM) HDFS cluster with a configured number of datanodes.
@@ -43,6 +53,10 @@
     private static final String DATA_PATH = "data/hdfs";
     private static final String HDFS_PATH = "/asterix";
     private static final HDFSCluster INSTANCE = new HDFSCluster();
+    //Temporary folder that holds generated binary files
+    private static final String BINARY_GEN_BASEDIR = "target" + File.separatorChar + "generated_bin_files";
+    //How many records should the schema inference method inspect to infer the schema for parquet files
+    private static final int NUM_OF_RECORDS_SCHEMA = 20;
 
     private MiniDFSCluster dfsCluster;
     private int numDataNodes = 2;
@@ -79,18 +93,25 @@
         build.startupOption(StartupOption.REGULAR);
         dfsCluster = build.build();
         dfs = FileSystem.get(conf);
-        loadData(basePath);
+        //Generate binary files from JSON files (e.g., parquet files)
+        generateBinaryFiles(basePath);
+        //Load JSON/ADM files to HDFS
+        loadData(basePath, DATA_PATH);
+        //Load generated binary files (e.g., parquet files) to HDFS
+        loadData(basePath, BINARY_GEN_BASEDIR);
     }
 
-    private void loadData(File localDataRoot) throws IOException {
+    private void loadData(File localDataRoot, String dataPath) throws IOException {
         Path destDir = new Path(HDFS_PATH);
         dfs.mkdirs(destDir);
-        File srcDir = new File(localDataRoot, DATA_PATH);
+        File srcDir = new File(localDataRoot, dataPath);
         if (srcDir.exists()) {
             File[] listOfFiles = srcDir.listFiles();
             for (File srcFile : listOfFiles) {
-                Path path = new Path(srcFile.getAbsolutePath());
-                dfs.copyFromLocalFile(path, destDir);
+                if (srcFile.isFile()) {
+                    Path path = new Path(srcFile.getAbsolutePath());
+                    dfs.copyFromLocalFile(path, destDir);
+                }
             }
         }
     }
@@ -102,6 +123,16 @@
         System.setProperty("hadoop.log.dir", "logs");
     }
 
+    private void generateBinaryFiles(File localDataRoot) throws IOException {
+        File srcPath = new File(localDataRoot, DATA_PATH);
+        File destPath = new File(localDataRoot, BINARY_GEN_BASEDIR);
+        //Delete old generated files
+        IoUtil.delete(destPath);
+        Files.createDirectory(Paths.get(destPath.getAbsolutePath()));
+        //Write parquet files
+        writeParquetDir(new File(srcPath, "parquet"), destPath);
+    }
+
     public void cleanup() throws Exception {
         if (dfsCluster != null) {
             dfsCluster.shutdown();
@@ -131,4 +162,27 @@
         return conf;
     }
 
+    private void writeParquetDir(File parquetSrcDir, File destPath) throws IOException {
+        File[] listOfFiles = parquetSrcDir.listFiles();
+        for (File jsonFile : listOfFiles) {
+            String fileName = jsonFile.getName().substring(0, jsonFile.getName().indexOf(".")) + ".parquet";
+            Path outputPath = new Path(destPath.getAbsolutePath(), fileName);
+            writeParquetFile(jsonFile, outputPath);
+        }
+    }
+
+    public void writeParquetFile(File jsonInputPath, Path parquetOutputPath) throws IOException {
+        final FileInputStream schemaInputStream = new FileInputStream(jsonInputPath);
+        final FileInputStream jsonInputStream = new FileInputStream(jsonInputPath);
+        //Infer Avro schema
+        final Schema inputSchema = JsonUtil.inferSchema(schemaInputStream, "parquet_schema", NUM_OF_RECORDS_SCHEMA);
+        try (JSONFileReader<Record> reader = new JSONFileReader<>(jsonInputStream, inputSchema, Record.class)) {
+            reader.initialize();
+            try (AvroParquetWriter<Record> writer = new AvroParquetWriter<>(parquetOutputPath, inputSchema)) {
+                for (Record record : reader) {
+                    writer.write(record);
+                }
+            }
+        }
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.1.ddl.sqlpp
new file mode 100644
index 0000000..8b1303f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.1.ddl.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Do not pushdown if the whole record was requested
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888"),
+  ("path"="/asterix/id_age.parquet"),
+  ("input-format"="parquet-input-format")
+);
+
+CREATE EXTERNAL DATASET ParquetDataset2(ParquetType) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888"),
+  ("path"="/asterix/id_age.parquet"),
+  ("input-format"="parquet-input-format")
+);
+
+CREATE EXTERNAL DATASET ParquetDataset3(ParquetType) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888"),
+  ("path"="/asterix/id_name_comment.parquet"),
+  ("input-format"="parquet-input-format")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.10.query.sqlpp
new file mode 100644
index 0000000..828b598
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.10.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Get only name and age
+* Expected Res : Success
+* Date         : August 30th 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+EXPLAIN
+SELECT p1.age, p2.name
+FROM ParquetDataset p1, ParquetDataset3 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.11.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.11.ddl.sqlpp
new file mode 100644
index 0000000..e9a8b3f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.11.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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  : Get common field
+* Expected Res : Success
+* Date         : August 30 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888"),
+  ("path"="/asterix/dummy_tweet.parquet"),
+  ("input-format"="parquet-input-format")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.12.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.12.update.sqlpp
new file mode 100644
index 0000000..c5796e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.12.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Get common field
+* Expected Res : Success
+* Date         : August 30 2020
+*/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.13.query.sqlpp
new file mode 100644
index 0000000..10d0788
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.13.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Get common field
+* Expected Res : Success
+* Date         : August 30 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT p.user.id, p.user.name
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.14.query.sqlpp
new file mode 100644
index 0000000..f726e8d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.14.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Get common field
+* Expected Res : Success
+* Date         : August 30 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+EXPLAIN
+SELECT p.user.id, p.user.name
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.2.update.sqlpp
new file mode 100644
index 0000000..8e71a42
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Do not pushdown if the whole record was requested
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.3.query.sqlpp
new file mode 100644
index 0000000..4ba4649
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Do not pushdown if the whole record was requested
+                 This case, the expression 'p.id' should not be pushed down
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.4.query.sqlpp
new file mode 100644
index 0000000..cea66ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.4.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Do not pushdown if the whole record was requested
+                 This case, the expression 'p.id' should not be pushed down
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.5.query.sqlpp
new file mode 100644
index 0000000..9492cdb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.5.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Do not pushdown if the whole record was requested
+                 This case, the expression 'p.id' should not be pushed down
+* Expected Res : Success
+* Date         : August 30th 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+EXPLAIN
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.6.query.sqlpp
new file mode 100644
index 0000000..d24c258
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.6.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Do not pushdown if the whole record was requested
+                 In this join, 'p2.id' should be pushed down, but not 'p1.id'
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SELECT p1, p2.id
+FROM ParquetDataset p1, ParquetDataset2 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.7.query.sqlpp
new file mode 100644
index 0000000..eb2d8e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.7.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Do not pushdown if the whole record was requested
+                 In this join, 'p2.id' should be pushed down, but not 'p1.id'
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT p1, p2.id
+FROM ParquetDataset p1, ParquetDataset2 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.8.query.sqlpp
new file mode 100644
index 0000000..4e60665
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.8.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Do not pushdown if the whole record was requested
+                 In this join, 'p2.id' should be pushed down, but not 'p1.id'
+* Expected Res : Success
+* Date         : August 30th 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+EXPLAIN
+SELECT p1, p2.id
+FROM ParquetDataset p1, ParquetDataset2 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.9.query.sqlpp
new file mode 100644
index 0000000..069c26a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/expression-pushdown/expression-pushdown.9.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Get only name and age
+* Expected Res : Success
+* Date         : August 30th 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT p1.age, p2.name
+FROM ParquetDataset p1, ParquetDataset3 p2
+WHERE p1.id = p2.id
+ORDER BY p2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..1e24678
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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  : Requesting non-existing fields should not fail
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888"),
+  ("path"="/asterix/dummy_tweet.parquet"),
+  ("input-format"="parquet-input-format")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.2.update.sqlpp
new file mode 100644
index 0000000..48ae519
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Requesting non-existing fields should not fail
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.3.query.sqlpp
new file mode 100644
index 0000000..1715bba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Requesting non-existing fields should not fail
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SELECT p.not_a_field1 IS MISSING as f1, p.user.not_a_field2 IS MISSING as f2
+FROM ParquetDataset p
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.4.query.sqlpp
new file mode 100644
index 0000000..6c030db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/missing-fields/missing-fields.4.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Requesting non-existing fields should not fail
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT p.not_a_field1 IS MISSING as f1, p.user.not_a_field2 IS MISSING as f2
+FROM ParquetDataset p
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp
new file mode 100644
index 0000000..7fc6176
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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  : Retrieve all fields from different Parquet files with different schemas
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888"),
+  ("path"="/asterix/id_name.parquet,/asterix/id_age.parquet"),
+  ("input-format"="parquet-input-format")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.2.update.sqlpp
new file mode 100644
index 0000000..a4fe4c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Retrieve all fields from different Parquet files with different schemas
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.3.query.sqlpp
new file mode 100644
index 0000000..910f20c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/multi-file-multi-schema/multi-file-multi-schema.3.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Retrieve all fields from different Parquet files with different schemas
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.1.ddl.sqlpp
new file mode 100644
index 0000000..db6c4d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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  : Concat two objects after pushdown
+* Expected Res : Success
+* Date         : September 22nd 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888"),
+  ("path"="/asterix/dummy_tweet.parquet"),
+  ("input-format"="parquet-input-format")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.2.update.sqlpp
new file mode 100644
index 0000000..334beeb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Concat two objects after pushdown
+* Expected Res : Success
+* Date         : September 22nd 2020
+*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.3.query.sqlpp
new file mode 100644
index 0000000..a429b73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Concat two objects after pushdown
+* Expected Res : Success
+* Date         : September 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+SELECT VALUE object_concat(p.coordinates, p.user).name
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.4.query.sqlpp
new file mode 100644
index 0000000..dd114e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.4.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Concat two objects after pushdown
+* Expected Res : Success
+* Date         : September 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+EXPLAIN
+SELECT VALUE object_concat(p.coordinates, p.user).name
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.5.query.sqlpp
new file mode 100644
index 0000000..06bd91c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.5.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Concat two objects after pushdown
+* Expected Res : Success
+* Date         : September 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+SELECT VALUE object_concat(p.coordinates, p.user)
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.6.query.sqlpp
new file mode 100644
index 0000000..0b42622
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/object-concat/object-concat.6.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Concat two objects after pushdown
+* Expected Res : Success
+* Date         : September 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+EXPLAIN
+SELECT VALUE object_concat(p.coordinates, p.user)
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..20454cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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  : Retrieve all fields from a Parquet file
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888"),
+  ("path"="/asterix/dummy_tweet.parquet"),
+  ("input-format"="parquet-input-format")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.2.update.sqlpp
new file mode 100644
index 0000000..6b0c1e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Retrieve all fields from a Parquet file
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.3.query.sqlpp
new file mode 100644
index 0000000..51a7b3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.3.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Retrieve all fields from a Parquet file
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.4.query.sqlpp
new file mode 100644
index 0000000..c04a65b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-all-fields/select-all-fields.4.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Retrieve all fields from a Parquet file
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT VALUE p
+FROM ParquetDataset p
+ORDER BY p.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.1.ddl.sqlpp
new file mode 100644
index 0000000..17ebc63
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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  : Retrieve the number of texts in all tweets
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888"),
+  ("path"="/asterix/dummy_tweet.parquet"),
+  ("input-format"="parquet-input-format")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.2.update.sqlpp
new file mode 100644
index 0000000..c39aabc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Retrieve the number of texts in all tweets
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.3.query.sqlpp
new file mode 100644
index 0000000..491d789
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Retrieve the number of texts in all tweets (with expression pushdown)
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT VALUE count(p.text)
+FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.4.query.sqlpp
new file mode 100644
index 0000000..84639bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Retrieve the number of texts in all tweets
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SELECT VALUE count(p.text)
+FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.5.query.sqlpp
new file mode 100644
index 0000000..dddb9b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Retrieve the number of users
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SELECT VALUE count(p.user.name)
+FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.6.query.sqlpp
new file mode 100644
index 0000000..76ac0ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/select-count-one-field/select-count-one-field.6.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Retrieve the number of users (with expression pushdown)
+* Expected Res : Success
+* Date         : June 22nd 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT VALUE count(p.user.name)
+FROM ParquetDataset p;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp
new file mode 100644
index 0000000..650e3a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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  : Test Standard UTF-8
+* Expected Res : Success
+* Date         : August 30th 2020
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE ParquetType as {
+};
+
+CREATE EXTERNAL DATASET ParquetDataset(ParquetType) USING hdfs
+(
+  ("hdfs"="hdfs://127.0.0.1:31888"),
+  ("path"="/asterix/id_name_comment.parquet"),
+  ("input-format"="parquet-input-format")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.2.update.sqlpp
new file mode 100644
index 0000000..01270f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Test Standard UTF-8
+* Expected Res : Success
+* Date         : August 30th 2020
+*/
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.3.query.sqlpp
new file mode 100644
index 0000000..f849a1f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.3.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Test Standard UTF-8
+* Expected Res : Success
+* Date         : August 30th 2020
+*/
+USE test;
+
+SELECT VALUE array_count(split(trim(p.comment),"𩸽"))
+FROM ParquetDataset p
+WHERE contains(p.comment, "𩸽");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.4.query.sqlpp
new file mode 100644
index 0000000..0267e54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/parquet/string-standard-utf8/string-standard-utf8.4.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Test Standard UTF-8
+* Expected Res : Success
+* Date         : August 30th 2020
+*/
+USE test;
+
+SET `compiler.external.field.pushdown` "true";
+
+SELECT VALUE array_count(split(trim(p.comment),"𩸽"))
+FROM ParquetDataset p
+WHERE contains(p.comment, "𩸽");
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index fc04b37..08532d6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -8,6 +8,7 @@
     "active\.memory\.global\.budget" : 67108864,
     "active\.stop\.timeout" : 3600,
     "active\.suspend\.timeout" : 3600,
+    "compiler\.external\.field\.pushdown" : false,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
     "compiler\.indexonly" : true,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index c4d58ed..5cded03 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -8,6 +8,7 @@
     "active\.memory\.global\.budget" : 67108864,
     "active\.stop\.timeout" : 3600,
     "active\.suspend\.timeout" : 3600,
+    "compiler\.external\.field\.pushdown" : false,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
     "compiler\.indexonly" : true,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 28b9e25..869e9c7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -8,6 +8,7 @@
     "active\.memory\.global\.budget" : 67108864,
     "active\.stop\.timeout" : 3600,
     "active\.suspend\.timeout" : 3600,
+    "compiler\.external\.field\.pushdown" : false,
     "compiler\.framesize" : 32768,
     "compiler\.groupmemory" : 163840,
     "compiler\.indexonly" : true,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.1.json
new file mode 100644
index 0000000..7e235c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.1.json
@@ -0,0 +1,7 @@
+{ "id": 8, "age": 10 }
+{ "id": 9, "age": 20 }
+{ "id": 10, "age": 30 }
+{ "id": 11, "age": 40 }
+{ "id": 12, "age": 50 }
+{ "id": 13, "age": 60 }
+{ "id": 14, "age": 70 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.10.adm
new file mode 100644
index 0000000..3c1ad9d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.10.adm
@@ -0,0 +1,32 @@
+distribute result [$$17]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    project ([$$17])
+    -- STREAM_PROJECT  |PARTITIONED|
+      assign [$$17] <- [{"id": $$21, "name": $$22}]
+      -- ASSIGN  |PARTITIONED|
+        project ([$$21, $$22])
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange
+          -- SORT_MERGE_EXCHANGE [$$20(ASC) ]  |PARTITIONED|
+            order (ASC, $$20)
+            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+              exchange
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$21, $$22, $$20])
+                -- STREAM_PROJECT  |PARTITIONED|
+                  assign [$$22, $$21] <- [$$19.getField("name"), $$19.getField("id")]
+                  -- ASSIGN  |PARTITIONED|
+                    project ([$$19, $$20])
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      assign [$$19, $$20] <- [$$p.getField("user"), $$p.getField("id")]
+                      -- ASSIGN  |PARTITIONED|
+                        exchange
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$p] <- test.ParquetDataset project (user, id)
+                          -- DATASOURCE_SCAN  |PARTITIONED|
+                            exchange
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              empty-tuple-source
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.2.json
new file mode 100644
index 0000000..7e235c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.2.json
@@ -0,0 +1,7 @@
+{ "id": 8, "age": 10 }
+{ "id": 9, "age": 20 }
+{ "id": 10, "age": 30 }
+{ "id": 11, "age": 40 }
+{ "id": 12, "age": 50 }
+{ "id": 13, "age": 60 }
+{ "id": 14, "age": 70 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.3.adm
new file mode 100644
index 0000000..5260a0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.3.adm
@@ -0,0 +1,22 @@
+distribute result [$$p]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    project ([$$p])
+    -- STREAM_PROJECT  |PARTITIONED|
+      exchange
+      -- SORT_MERGE_EXCHANGE [$$13(ASC) ]  |PARTITIONED|
+        order (ASC, $$13)
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          exchange
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            assign [$$13] <- [$$p.getField("id")]
+            -- ASSIGN  |PARTITIONED|
+              exchange
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                data-scan []<-[$$p] <- test.ParquetDataset
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  exchange
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    empty-tuple-source
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.4.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.4.json
new file mode 100644
index 0000000..88b6965
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.4.json
@@ -0,0 +1,7 @@
+{ "p1": { "id": 8, "age": 10 }, "id": 8 }
+{ "p1": { "id": 9, "age": 20 }, "id": 9 }
+{ "p1": { "id": 10, "age": 30 }, "id": 10 }
+{ "p1": { "id": 11, "age": 40 }, "id": 11 }
+{ "p1": { "id": 12, "age": 50 }, "id": 12 }
+{ "p1": { "id": 13, "age": 60 }, "id": 13 }
+{ "p1": { "id": 14, "age": 70 }, "id": 14 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.5.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.5.json
new file mode 100644
index 0000000..88b6965
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.5.json
@@ -0,0 +1,7 @@
+{ "p1": { "id": 8, "age": 10 }, "id": 8 }
+{ "p1": { "id": 9, "age": 20 }, "id": 9 }
+{ "p1": { "id": 10, "age": 30 }, "id": 10 }
+{ "p1": { "id": 11, "age": 40 }, "id": 11 }
+{ "p1": { "id": 12, "age": 50 }, "id": 12 }
+{ "p1": { "id": 13, "age": 60 }, "id": 13 }
+{ "p1": { "id": 14, "age": 70 }, "id": 14 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.6.adm
new file mode 100644
index 0000000..d22d967
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.6.adm
@@ -0,0 +1,46 @@
+distribute result [$$28]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    project ([$$28])
+    -- STREAM_PROJECT  |PARTITIONED|
+      assign [$$28] <- [{"p1": $$p1, "id": $$30}]
+      -- ASSIGN  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$30(ASC) ]  |PARTITIONED|
+          order (ASC, $$30)
+          -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              project ([$$p1, $$30])
+              -- STREAM_PROJECT  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  join (eq($$31, $$30))
+                  -- HYBRID_HASH_JOIN [$$31][$$30]  |PARTITIONED|
+                    exchange
+                    -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                      assign [$$31] <- [$$p1.getField("id")]
+                      -- ASSIGN  |PARTITIONED|
+                        exchange
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          data-scan []<-[$$p1] <- test.ParquetDataset
+                          -- DATASOURCE_SCAN  |PARTITIONED|
+                            exchange
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              empty-tuple-source
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    exchange
+                    -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
+                      project ([$$30])
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        assign [$$30] <- [$$p2.getField("id")]
+                        -- ASSIGN  |PARTITIONED|
+                          exchange
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            data-scan []<-[$$p2] <- test.ParquetDataset2 project (id)
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              exchange
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                empty-tuple-source
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.7.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.7.json
new file mode 100644
index 0000000..aa3f801
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.7.json
@@ -0,0 +1 @@
+{ "age": 10, "name": "William" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.8.adm
new file mode 100644
index 0000000..afd33c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.8.adm
@@ -0,0 +1,50 @@
+distribute result [$$29]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    project ([$$29])
+    -- STREAM_PROJECT  |PARTITIONED|
+      assign [$$29] <- [{"age": $$34, "name": $$35}]
+      -- ASSIGN  |PARTITIONED|
+        project ([$$34, $$35])
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange
+          -- SORT_MERGE_EXCHANGE [$$32(ASC) ]  |PARTITIONED|
+            order (ASC, $$32)
+            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+              exchange
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$34, $$35, $$32])
+                -- STREAM_PROJECT  |PARTITIONED|
+                  exchange
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    join (eq($$31, $$32))
+                    -- HYBRID_HASH_JOIN [$$31][$$32]  |PARTITIONED|
+                      exchange
+                      -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                        project ([$$34, $$31])
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          assign [$$34, $$31] <- [$$p1.getField("age"), $$p1.getField("id")]
+                          -- ASSIGN  |PARTITIONED|
+                            exchange
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$p1] <- test.ParquetDataset project (age, id)
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                exchange
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  empty-tuple-source
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      exchange
+                      -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                        project ([$$35, $$32])
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          assign [$$35, $$32] <- [$$p2.getField("name"), $$p2.getField("id")]
+                          -- ASSIGN  |PARTITIONED|
+                            exchange
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              data-scan []<-[$$p2] <- test.ParquetDataset3 project (name, id)
+                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                exchange
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  empty-tuple-source
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.9.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.9.json
new file mode 100644
index 0000000..a1ad24e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/expression-pushdown/expression-pushdown.9.json
@@ -0,0 +1,2 @@
+{ "id": 1, "name": "string" }
+{ "id": 1, "name": "string" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.1.json
new file mode 100644
index 0000000..8876910
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.1.json
@@ -0,0 +1,2 @@
+{ "f1": true, "f2": true }
+{ "f1": true, "f2": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.2.json
new file mode 100644
index 0000000..8876910
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/missing-fields/missing-fields.2.json
@@ -0,0 +1,2 @@
+{ "f1": true, "f2": true }
+{ "f1": true, "f2": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/multi-file-multi-schema/multi-file-multi-schema.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/multi-file-multi-schema/multi-file-multi-schema.1.json
new file mode 100644
index 0000000..0e2b980
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/multi-file-multi-schema/multi-file-multi-schema.1.json
@@ -0,0 +1,14 @@
+{ "id": 1, "name": "John" }
+{ "id": 2, "name": "Abel" }
+{ "id": 3, "name": "Sandy" }
+{ "id": 4, "name": "Alex" }
+{ "id": 5, "name": "Mike" }
+{ "id": 6, "name": "Tom" }
+{ "id": 7, "name": "Jerry" }
+{ "id": 8, "age": 10 }
+{ "id": 9, "age": 20 }
+{ "id": 10, "age": 30 }
+{ "id": 11, "age": 40 }
+{ "id": 12, "age": 50 }
+{ "id": 13, "age": 60 }
+{ "id": 14, "age": 70 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.1.json
new file mode 100644
index 0000000..1b425f7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.1.json
@@ -0,0 +1,2 @@
+"string"
+"string"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.2.adm
new file mode 100644
index 0000000..a097e74
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.2.adm
@@ -0,0 +1,24 @@
+distribute result [$$16]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    project ([$$16])
+    -- STREAM_PROJECT  |PARTITIONED|
+      exchange
+      -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+        order (ASC, $$18)
+        -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+          exchange
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            project ([$$16, $$18])
+            -- STREAM_PROJECT  |PARTITIONED|
+              assign [$$16, $$18] <- [object-concat($$p.getField("coordinates"), $$p.getField("user")).getField("name"), $$p.getField("id")]
+              -- ASSIGN  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  data-scan []<-[$$p] <- test.ParquetDataset project (coordinates, user, id)
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    exchange
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      empty-tuple-source
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.3.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.3.json
new file mode 100644
index 0000000..e22e026
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.3.json
@@ -0,0 +1,2 @@
+{ "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "location": "string", "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "time_zone": "string", "url": "string", "utc_offset": 1, "verified": true, "coordinates": [ 1.1 ], "type": "string" }
+{ "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "location": "string", "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "time_zone": "string", "url": "string", "utc_offset": 1, "verified": true, "coordinates": [ 1.1 ], "type": "string" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.4.adm
new file mode 100644
index 0000000..a5f69e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/object-concat/object-concat.4.adm
@@ -0,0 +1,28 @@
+distribute result [$$15]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    project ([$$15])
+    -- STREAM_PROJECT  |PARTITIONED|
+      assign [$$15] <- [object-concat($$18, $$19)]
+      -- ASSIGN  |PARTITIONED|
+        project ([$$18, $$19])
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange
+          -- SORT_MERGE_EXCHANGE [$$17(ASC) ]  |PARTITIONED|
+            order (ASC, $$17)
+            -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+              exchange
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                project ([$$18, $$19, $$17])
+                -- STREAM_PROJECT  |PARTITIONED|
+                  assign [$$19, $$18, $$17] <- [$$p.getField("user"), $$p.getField("coordinates"), $$p.getField("id")]
+                  -- ASSIGN  |PARTITIONED|
+                    exchange
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$p] <- test.ParquetDataset project (user, coordinates, id)
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        exchange
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          empty-tuple-source
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.1.json
new file mode 100644
index 0000000..53f2518
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.1.json
@@ -0,0 +1,2 @@
+{ "coordinates": { "coordinates": [ 1.1 ], "type": "string" }, "created_at": "string", "entities": { "urls": [ { "display_url": "string", "expanded_url": "string", "indices": [ 1 ], "url": "string" } ], "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "geo": { "coordinates": [ 1.1 ], "type": "string" }, "id": "0000000", "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "place": { "bounding_box": { "coordinates": [ [ [ 1.1 ] ] ], "type": "string" }, "country": "string", "country_code": "string", "full_name": "string", "id": "string", "name": "string", "place_type": "string", "url": "string" }, "possibly_sensitive": true, "quoted_status": { "created_at": "string", "entities": { "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "id": 1, "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "verified": true } }, "quoted_status_id": 1, "quoted_status_id_str": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "timestamp_ms": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "location": "string", "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "time_zone": "string", "url": "string", "utc_offset": 1, "verified": true } }
+{ "coordinates": { "coordinates": [ 1.1 ], "type": "string" }, "created_at": "string", "favorite_count": 1, "favorited": true, "filter_level": "string", "geo": { "coordinates": [ 1.1 ], "type": "string" }, "id": "11111111111111111111", "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "place": { "bounding_box": { "coordinates": [ [ [ 1.1 ] ] ], "type": "string" }, "country": "string", "country_code": "string", "full_name": "string", "id": "string", "name": "string", "place_type": "string", "url": "string" }, "possibly_sensitive": true, "quoted_status": { "created_at": "string", "entities": { "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "id": 1, "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "verified": true } }, "quoted_status_id": 1, "quoted_status_id_str": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "timestamp_ms": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "location": "string", "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "time_zone": "string", "url": "string", "utc_offset": 1, "verified": true } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.2.json
new file mode 100644
index 0000000..53f2518
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-all-fields/select-all-fields.2.json
@@ -0,0 +1,2 @@
+{ "coordinates": { "coordinates": [ 1.1 ], "type": "string" }, "created_at": "string", "entities": { "urls": [ { "display_url": "string", "expanded_url": "string", "indices": [ 1 ], "url": "string" } ], "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "geo": { "coordinates": [ 1.1 ], "type": "string" }, "id": "0000000", "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "place": { "bounding_box": { "coordinates": [ [ [ 1.1 ] ] ], "type": "string" }, "country": "string", "country_code": "string", "full_name": "string", "id": "string", "name": "string", "place_type": "string", "url": "string" }, "possibly_sensitive": true, "quoted_status": { "created_at": "string", "entities": { "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "id": 1, "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "verified": true } }, "quoted_status_id": 1, "quoted_status_id_str": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "timestamp_ms": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "location": "string", "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "time_zone": "string", "url": "string", "utc_offset": 1, "verified": true } }
+{ "coordinates": { "coordinates": [ 1.1 ], "type": "string" }, "created_at": "string", "favorite_count": 1, "favorited": true, "filter_level": "string", "geo": { "coordinates": [ 1.1 ], "type": "string" }, "id": "11111111111111111111", "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "place": { "bounding_box": { "coordinates": [ [ [ 1.1 ] ] ], "type": "string" }, "country": "string", "country_code": "string", "full_name": "string", "id": "string", "name": "string", "place_type": "string", "url": "string" }, "possibly_sensitive": true, "quoted_status": { "created_at": "string", "entities": { "user_mentions": [ { "id": 1, "id_str": "string", "indices": [ 1 ], "name": "string", "screen_name": "string" } ] }, "favorite_count": 1, "favorited": true, "filter_level": "string", "id": 1, "id_str": "string", "in_reply_to_screen_name": "string", "in_reply_to_status_id": 1, "in_reply_to_status_id_str": "string", "in_reply_to_user_id": 1, "in_reply_to_user_id_str": "string", "is_quote_status": true, "lang": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "verified": true } }, "quoted_status_id": 1, "quoted_status_id_str": "string", "retweet_count": 1, "retweeted": true, "source": "string", "text": "string", "timestamp_ms": "string", "truncated": true, "user": { "contributors_enabled": true, "created_at": "string", "default_profile": true, "default_profile_image": true, "description": "string", "favourites_count": 1, "followers_count": 1, "friends_count": 1, "geo_enabled": true, "id": 1, "id_str": "string", "is_translator": true, "lang": "string", "listed_count": 1, "location": "string", "name": "string", "profile_background_color": "string", "profile_background_image_url": "string", "profile_background_image_url_https": "string", "profile_background_tile": true, "profile_banner_url": "string", "profile_image_url": "string", "profile_image_url_https": "string", "profile_link_color": "string", "profile_sidebar_border_color": "string", "profile_sidebar_fill_color": "string", "profile_text_color": "string", "profile_use_background_image": true, "protected": true, "screen_name": "string", "statuses_count": 1, "time_zone": "string", "url": "string", "utc_offset": 1, "verified": true } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.1.json
@@ -0,0 +1 @@
+2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.2.json
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.2.json
@@ -0,0 +1 @@
+2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.3.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.3.json
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.3.json
@@ -0,0 +1 @@
+2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.4.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.4.json
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/select-count-one-field/select-count-one-field.4.json
@@ -0,0 +1 @@
+2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.1.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.1.json
new file mode 100644
index 0000000..6fb86be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.1.json
@@ -0,0 +1,2 @@
+2
+301
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.2.json b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.2.json
new file mode 100644
index 0000000..6fb86be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/parquet/string-standard-utf8/string-standard-utf8.2.json
@@ -0,0 +1,2 @@
+2
+301
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
index 45b36d0..5dfca79 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_hdfs.xml
@@ -157,4 +157,41 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="parquet">
+    <test-case FilePath="parquet">
+      <compilation-unit name="select-all-fields">
+        <output-dir compare="Text">select-all-fields</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="parquet">
+      <compilation-unit name="select-count-one-field">
+        <output-dir compare="Text">select-count-one-field</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="parquet">
+      <compilation-unit name="multi-file-multi-schema">
+        <output-dir compare="Text">multi-file-multi-schema</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="parquet">
+      <compilation-unit name="expression-pushdown">
+        <output-dir compare="Text">expression-pushdown</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="parquet">
+      <compilation-unit name="missing-fields">
+        <output-dir compare="Text">missing-fields</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="parquet">
+      <compilation-unit name="string-standard-utf8">
+        <output-dir compare="Text">string-standard-utf8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="parquet">
+      <compilation-unit name="object-concat">
+        <output-dir compare="Text">object-concat</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
 </test-suite>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 9ec6279..442a3e0 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -67,16 +67,23 @@
                         + "other integer values dictate the number of query execution parallel partitions. The system will "
                         + "fall back to use the number of all available CPU cores in the cluster as the degree of parallelism "
                         + "if the number set by a user is too large or too small"),
-        COMPILER_SORT_PARALLEL(BOOLEAN, AlgebricksConfig.SORT_PARALLEL, "Enabling/Disabling full parallel sort"),
+        COMPILER_SORT_PARALLEL(
+                BOOLEAN,
+                AlgebricksConfig.SORT_PARALLEL_DEFAULT,
+                "Enabling/Disabling full parallel sort"),
         COMPILER_SORT_SAMPLES(
                 POSITIVE_INTEGER,
-                AlgebricksConfig.SORT_SAMPLES,
+                AlgebricksConfig.SORT_SAMPLES_DEFAULT,
                 "The number of samples which parallel sorting should take from each partition"),
         COMPILER_INDEXONLY(BOOLEAN, AlgebricksConfig.INDEX_ONLY_DEFAULT, "Enabling/disabling index-only plans"),
         COMPILER_INTERNAL_SANITYCHECK(
                 BOOLEAN,
                 AlgebricksConfig.SANITYCHECK_DEFAULT,
-                "Enable/disable compiler sanity check");
+                "Enable/disable compiler sanity check"),
+        COMPILER_EXTERNAL_FIELD_PUSHDOWN(
+                BOOLEAN,
+                AlgebricksConfig.EXTERNAL_FIELD_PUSHDOWN_DEFAULT,
+                "Enable pushdown of field accesses to the external dataset data-scan operator");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -129,6 +136,8 @@
 
     public static final String COMPILER_INTERNAL_SANITYCHECK_KEY = Option.COMPILER_INTERNAL_SANITYCHECK.ini();
 
+    public static final String COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY = Option.COMPILER_EXTERNAL_FIELD_PUSHDOWN.ini();
+
     public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
 
     public CompilerProperties(PropertiesAccessor accessor) {
@@ -178,4 +187,8 @@
     public boolean isSanityCheck() {
         return accessor.getBoolean(Option.COMPILER_INTERNAL_SANITYCHECK);
     }
+
+    public boolean isFieldAccessPushdown() {
+        return accessor.getBoolean(Option.COMPILER_EXTERNAL_FIELD_PUSHDOWN);
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
index 4f2a124..0402b97 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
@@ -64,6 +64,8 @@
                 compilerProperties.isIndexOnly());
         boolean sanityCheck = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
                 compilerProperties.isSanityCheck());
+        boolean externalFieldPushdown = getBoolean(querySpecificConfig,
+                CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, compilerProperties.isFieldAccessPushdown());
 
         PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
         physOptConf.setFrameSize(frameSize);
@@ -76,6 +78,7 @@
         physOptConf.setSortSamples(sortNumSamples);
         physOptConf.setIndexOnly(indexOnly);
         physOptConf.setSanityCheckEnabled(sanityCheck);
+        physOptConf.setExternalFieldPushdown(externalFieldPushdown);
         return physOptConf;
     }
 
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index 37c8a61..a3daba8 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -158,7 +158,7 @@
     </plugins>
     <pluginManagement>
       <plugins>
-                <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+        <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
         <plugin>
           <groupId>org.eclipse.m2e</groupId>
           <artifactId>lifecycle-mapping</artifactId>
@@ -465,8 +465,16 @@
       <artifactId>azure-storage-blob</artifactId>
     </dependency>
     <dependency>
-        <groupId>org.msgpack</groupId>
-        <artifactId>msgpack-core</artifactId>
+      <groupId>org.msgpack</groupId>
+      <artifactId>msgpack-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-column</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-hadoop</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.logging.log4j</groupId>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
index dba0f27..f436aa1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
@@ -35,6 +35,7 @@
 import org.apache.asterix.external.indexing.IndexingScheduler;
 import org.apache.asterix.external.input.record.reader.IndexingStreamRecordReader;
 import org.apache.asterix.external.input.record.reader.hdfs.HDFSRecordReader;
+import org.apache.asterix.external.input.record.reader.hdfs.parquet.ParquetFileRecordReader;
 import org.apache.asterix.external.input.record.reader.stream.StreamRecordReader;
 import org.apache.asterix.external.input.stream.HDFSInputStream;
 import org.apache.asterix.external.provider.ExternalIndexerProvider;
@@ -107,7 +108,8 @@
             read = new boolean[readSchedule.length];
             Arrays.fill(read, false);
             String formatString = configuration.get(ExternalDataConstants.KEY_FORMAT);
-            if (formatString == null || formatString.equals(ExternalDataConstants.FORMAT_HDFS_WRITABLE)) {
+            if (formatString == null || formatString.equals(ExternalDataConstants.FORMAT_HDFS_WRITABLE)
+                    || formatString.equals(ExternalDataConstants.FORMAT_NOOP)) {
                 RecordReader<?, ?> reader = conf.getInputFormat().getRecordReader(inputSplits[0], conf, Reporter.NULL);
                 this.recordClass = reader.createValue().getClass();
                 reader.close();
@@ -215,7 +217,7 @@
                 }
             }
             restoreConfig(ctx);
-            return new HDFSRecordReader<>(read, inputSplits, readSchedule, nodeName, conf, files, indexer);
+            return createRecordReader(configuration, read, inputSplits, readSchedule, nodeName, conf, files, indexer);
         } catch (Exception e) {
             throw HyracksDataException.create(e);
         }
@@ -240,4 +242,15 @@
     public List<String> getRecordReaderNames() {
         return recordReaderNames;
     }
+
+    private static IRecordReader<? extends Object> createRecordReader(Map<String, String> configuration, boolean[] read,
+            InputSplit[] inputSplits, String[] readSchedule, String nodeName, JobConf conf, List<ExternalFile> files,
+            IExternalIndexer indexer) throws IOException {
+        if (configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT.trim())
+                .equals(ExternalDataConstants.INPUT_FORMAT_PARQUET)) {
+            return new ParquetFileRecordReader<>(read, inputSplits, readSchedule, nodeName, conf);
+        } else {
+            return new HDFSRecordReader<>(read, inputSplits, readSchedule, nodeName, conf, files, indexer);
+        }
+    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/ValueReferenceRecord.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/ValueReferenceRecord.java
new file mode 100644
index 0000000..49ce643
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/ValueReferenceRecord.java
@@ -0,0 +1,51 @@
+/*
+ * 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.input.record;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public class ValueReferenceRecord<V extends IValueReference> implements IRawRecord<V> {
+    private V record;
+
+    @Override
+    public byte[] getBytes() {
+        return record.getByteArray();
+    }
+
+    @Override
+    public int size() {
+        return record.getLength();
+    }
+
+    @Override
+    public V get() {
+        return record;
+    }
+
+    @Override
+    public void reset() {
+        //no-op
+    }
+
+    @Override
+    public void set(V t) {
+        record = t;
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSRecordReader.java
new file mode 100644
index 0000000..0d498b8
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/AbstractHDFSRecordReader.java
@@ -0,0 +1,165 @@
+/*
+ * 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.input.record.reader.hdfs;
+
+import java.io.IOException;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordReader;
+import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
+import org.apache.asterix.external.input.record.GenericRecord;
+import org.apache.asterix.external.util.FeedLogManager;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+
+public abstract class AbstractHDFSRecordReader<K, V> implements IRecordReader<V> {
+    protected RecordReader<K, V> reader;
+    protected V value = null;
+    protected K key = null;
+    protected int currentSplitIndex = 0;
+    protected boolean[] read;
+    protected InputFormat<?, ?> inputFormat;
+    protected InputSplit[] inputSplits;
+    protected String[] readSchedule;
+    protected String nodeName;
+    protected JobConf conf;
+    protected IRawRecord<V> record;
+    private boolean firstInputSplit;
+
+    public AbstractHDFSRecordReader(boolean[] read, InputSplit[] inputSplits, String[] readSchedule, String nodeName,
+            JobConf conf) {
+        this.read = read;
+        this.inputSplits = inputSplits;
+        this.readSchedule = readSchedule;
+        this.nodeName = nodeName;
+        this.conf = conf;
+        this.inputFormat = conf.getInputFormat();
+        this.record = new GenericRecord<>();
+        reader = new EmptyRecordReader<>();
+        firstInputSplit = false;
+    }
+
+    public AbstractHDFSRecordReader(boolean[] read, InputSplit[] inputSplits, String[] readSchedule, String nodeName,
+            IRawRecord<V> record, JobConf conf) {
+        this.read = read;
+        this.inputSplits = inputSplits;
+        this.readSchedule = readSchedule;
+        this.nodeName = nodeName;
+        this.conf = conf;
+        this.inputFormat = conf.getInputFormat();
+        this.record = record;
+        reader = new EmptyRecordReader<>();
+        firstInputSplit = false;
+    }
+
+    @Override
+    public void close() throws IOException {
+        reader.close();
+    }
+
+    @Override
+    public boolean hasNext() throws Exception {
+        if (!firstInputSplit) {
+            firstInputSplit = true;
+            nextInputSplit();
+        }
+
+        if (reader.next(key, value)) {
+            return true;
+        }
+        while (nextInputSplit()) {
+            if (reader.next(key, value)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public IRawRecord<V> next() throws IOException {
+        record.set(value);
+        return record;
+    }
+
+    private boolean nextInputSplit() throws IOException {
+        for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+            /**
+             * read all the partitions scheduled to the current node
+             */
+            if (readSchedule[currentSplitIndex].equals(nodeName)) {
+                /**
+                 * pick an unread split to read synchronize among
+                 * simultaneous partitions in the same machine
+                 */
+                boolean skipToNextInputsplit = false;
+                synchronized (read) {
+                    if (!read[currentSplitIndex]) {
+                        read[currentSplitIndex] = true;
+                    } else {
+                        skipToNextInputsplit = true;
+                    }
+                }
+
+                skipToNextInputsplit |= onNextInputSplit();
+
+                if (skipToNextInputsplit) {
+                    continue;
+                }
+
+                reader.close();
+                reader = getRecordReader(currentSplitIndex);
+                return true;
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Returns true if need to go to next split without closing the current reader
+     *
+     * @throws IOException
+     */
+    protected abstract boolean onNextInputSplit() throws IOException;
+
+    protected abstract RecordReader<K, V> getRecordReader(int splitIndex) throws IOException;
+
+    @Override
+    public boolean stop() {
+        return false;
+    }
+
+    public RecordReader<K, V> getReader() {
+        return reader;
+    }
+
+    @Override
+    public void setFeedLogManager(FeedLogManager feedLogManager) {
+    }
+
+    @Override
+    public void setController(AbstractFeedDataFlowController controller) {
+    }
+
+    @Override
+    public boolean handleException(Throwable th) {
+        return false;
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java
index 5b3d04f..9fbc800 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSRecordReader.java
@@ -23,123 +23,40 @@
 
 import org.apache.asterix.external.api.IExternalIndexer;
 import org.apache.asterix.external.api.IIndexingDatasource;
-import org.apache.asterix.external.api.IRawRecord;
-import org.apache.asterix.external.api.IRecordReader;
-import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
 import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.input.record.GenericRecord;
-import org.apache.asterix.external.util.FeedLogManager;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class HDFSRecordReader<K, V extends Writable> implements IRecordReader<Writable>, IIndexingDatasource {
-
-    protected RecordReader<K, Writable> reader;
-    protected V value = null;
-    protected K key = null;
-    protected int currentSplitIndex = 0;
-    protected boolean read[];
-    protected InputFormat<?, ?> inputFormat;
-    protected InputSplit[] inputSplits;
-    protected String[] readSchedule;
-    protected String nodeName;
-    protected JobConf conf;
-    protected GenericRecord<Writable> record;
+public class HDFSRecordReader<K, V extends Writable> extends AbstractHDFSRecordReader<K, V>
+        implements IIndexingDatasource {
     // Indexing variables
-    protected final IExternalIndexer indexer;
-    protected final List<ExternalFile> snapshot;
-    protected final FileSystem hdfs;
+    private final IExternalIndexer indexer;
+    private final List<ExternalFile> snapshot;
+    private final FileSystem hdfs;
 
-    public HDFSRecordReader(boolean read[], InputSplit[] inputSplits, String[] readSchedule, String nodeName,
+    public HDFSRecordReader(boolean[] read, InputSplit[] inputSplits, String[] readSchedule, String nodeName,
             JobConf conf, List<ExternalFile> snapshot, IExternalIndexer indexer) throws IOException {
-        this.read = read;
-        this.inputSplits = inputSplits;
-        this.readSchedule = readSchedule;
-        this.nodeName = nodeName;
-        this.conf = conf;
-        this.inputFormat = conf.getInputFormat();
-        this.reader = new EmptyRecordReader<K, Writable>();
-        this.record = new GenericRecord<Writable>();
+        super(read, inputSplits, readSchedule, nodeName, conf);
         this.indexer = indexer;
         this.snapshot = snapshot;
         this.hdfs = FileSystem.get(conf);
-        nextInputSplit();
-    }
-
-    @Override
-    public void close() throws IOException {
-        reader.close();
-    }
-
-    @Override
-    public boolean hasNext() throws Exception {
-        if (reader.next(key, value)) {
-            return true;
-        }
-        while (nextInputSplit()) {
-            if (reader.next(key, value)) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    @Override
-    public IRawRecord<Writable> next() throws IOException {
-        record.set(value);
-        return record;
-    }
-
-    private boolean nextInputSplit() throws IOException {
-        for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-            /**
-             * read all the partitions scheduled to the current node
-             */
-            if (readSchedule[currentSplitIndex].equals(nodeName)) {
-                /**
-                 * pick an unread split to read synchronize among
-                 * simultaneous partitions in the same machine
-                 */
-                synchronized (read) {
-                    if (read[currentSplitIndex] == false) {
-                        read[currentSplitIndex] = true;
-                    } else {
-                        continue;
-                    }
-                }
-                if (snapshot != null) {
-                    String fileName = ((FileSplit) (inputSplits[currentSplitIndex])).getPath().toUri().getPath();
-                    FileStatus fileStatus = hdfs.getFileStatus(new Path(fileName));
-                    // Skip if not the same file stored in the files snapshot
-                    if (fileStatus.getModificationTime() != snapshot.get(currentSplitIndex).getLastModefiedTime()
-                            .getTime()) {
-                        continue;
-                    }
-                }
-
-                reader.close();
-                reader = getRecordReader(currentSplitIndex);
-                return true;
-            }
-        }
-        return false;
     }
 
     @SuppressWarnings("unchecked")
-    private RecordReader<K, Writable> getRecordReader(int splitIndex) throws IOException {
-        reader = (RecordReader<K, Writable>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
+    @Override
+    protected RecordReader<K, V> getRecordReader(int splitIndex) throws IOException {
+        reader = (RecordReader<K, V>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
         if (key == null) {
             key = reader.createKey();
-            value = (V) reader.createValue();
+            value = reader.createValue();
         }
         if (indexer != null) {
             try {
@@ -152,6 +69,19 @@
     }
 
     @Override
+    protected boolean onNextInputSplit() throws IOException {
+        if (snapshot != null) {
+            String fileName = ((FileSplit) (inputSplits[currentSplitIndex])).getPath().toUri().getPath();
+            FileStatus fileStatus = hdfs.getFileStatus(new Path(fileName));
+            // Skip if not the same file stored in the files snapshot
+            if (fileStatus.getModificationTime() != snapshot.get(currentSplitIndex).getLastModefiedTime().getTime()) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
     public boolean stop() {
         return false;
     }
@@ -171,20 +101,8 @@
         return currentSplitIndex;
     }
 
-    public RecordReader<K, Writable> getReader() {
+    @Override
+    public RecordReader<K, V> getReader() {
         return reader;
     }
-
-    @Override
-    public void setFeedLogManager(FeedLogManager feedLogManager) {
-    }
-
-    @Override
-    public void setController(AbstractFeedDataFlowController controller) {
-    }
-
-    @Override
-    public boolean handleException(Throwable th) {
-        return false;
-    }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AbstractComplexConverter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AbstractComplexConverter.java
new file mode 100644
index 0000000..81b61f8
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AbstractComplexConverter.java
@@ -0,0 +1,153 @@
+/*
+ * 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.input.record.reader.hdfs.parquet;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.external.parser.jackson.ParserContext;
+import org.apache.hyracks.data.std.api.IMutableValueStorage;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.io.api.Converter;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.Type;
+import org.apache.parquet.schema.Type.Repetition;
+
+abstract class AbstractComplexConverter extends GroupConverter implements IFieldValue {
+    protected final AbstractComplexConverter parent;
+    private final IValueReference fieldName;
+    private final int index;
+    private final Converter[] converters;
+    protected final ParserContext context;
+    protected IMutableValueStorage tempStorage;
+
+    public AbstractComplexConverter(AbstractComplexConverter parent, int index, GroupType parquetType,
+            ParserContext context) {
+        this(parent, null, index, parquetType, context);
+    }
+
+    public AbstractComplexConverter(AbstractComplexConverter parent, IValueReference fieldName, int index,
+            GroupType parquetType, ParserContext context) {
+        this.parent = parent;
+        this.fieldName = fieldName;
+        this.index = index;
+        this.context = context;
+        converters = new Converter[parquetType.getFieldCount()];
+        for (int i = 0; i < parquetType.getFieldCount(); i++) {
+            final Type type = parquetType.getType(i);
+            if (type.isPrimitive()) {
+                converters[i] = createAtomicConverter(parquetType, i);
+            } else if (type.getOriginalType() == OriginalType.LIST) {
+                converters[i] = createArrayConverter(parquetType, i);
+            } else if (type.getRepetition() == Repetition.REPEATED) {
+                converters[i] = createRepeatedConverter(parquetType, i);
+            } else {
+                converters[i] = createObjectConverter(parquetType, i);
+            }
+        }
+    }
+
+    /**
+     * Add child value (the caller is the child itself)
+     *
+     * @param value Child value
+     */
+    protected abstract void addValue(IFieldValue value);
+
+    protected abstract AtomicConverter createAtomicConverter(GroupType type, int index);
+
+    protected abstract ArrayConverter createArrayConverter(GroupType type, int index);
+
+    protected abstract ObjectConverter createObjectConverter(GroupType type, int index);
+
+    /**
+     * Parquet file created by (old) Avro writer treat repeated values differently from files created by Spark.
+     * Example:
+     * Let us consider the object <pre>{"urls":[{"display_url": "string", "expanded_url": "string"}]}</pre>
+     *
+     * @formatter:off
+     *
+     * In Avro:
+     * optional group urls (LIST) {
+     *    // if number of fields > 1, then should be treated as array of objects
+     *    repeated group array {
+     *       optional binary display_url (UTF8);
+     *       optional binary expanded_url (UTF8);
+     *    }
+     * }
+     *
+     * In Spark:
+     * optional group urls (LIST) {
+     *    repeated group list {
+     *       // Similar to JSON, the object fields are placed in an inner group
+     *       optional group item {
+     *          optional binary display_url (UTF8);
+     *          optional binary expanded_url (UTF8);
+     *       }
+     *    }
+     * }
+     *
+     * @formatter:on
+     */
+    public AbstractComplexConverter createRepeatedConverter(GroupType type, int index) {
+        final GroupType arrayType = type.getType(index).asGroupType();
+        if (arrayType.getFieldCount() == 1) {
+            return new RepeatedConverter(this, index, arrayType, context);
+        }
+        return new ObjectConverter(this, index, arrayType, context);
+    }
+
+    @Override
+    public IValueReference getFieldName() {
+        return fieldName;
+    }
+
+    @Override
+    public int getIndex() {
+        return index;
+    }
+
+    @Override
+    public Converter getConverter(int fieldIndex) {
+        return converters[fieldIndex];
+    }
+
+    protected DataOutput getDataOutput() {
+        tempStorage.reset();
+        return tempStorage.getDataOutput();
+    }
+
+    protected IMutableValueStorage getValue() {
+        return tempStorage;
+    }
+
+    protected DataOutput getParentDataOutput() {
+        return parent.getDataOutput();
+    }
+
+    protected void addThisValueToParent() {
+        if (parent == null) {
+            //root
+            return;
+        }
+        parent.addValue(this);
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ArrayConverter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ArrayConverter.java
new file mode 100644
index 0000000..9e8da77
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ArrayConverter.java
@@ -0,0 +1,86 @@
+/*
+ * 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.input.record.reader.hdfs.parquet;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.external.parser.jackson.ParserContext;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.schema.GroupType;
+
+class ArrayConverter extends AbstractComplexConverter {
+    private IAsterixListBuilder builder;
+
+    public ArrayConverter(AbstractComplexConverter parent, int index, GroupType parquetType, ParserContext context) {
+        super(parent, index, parquetType, context);
+    }
+
+    public ArrayConverter(AbstractComplexConverter parent, IValueReference fieldName, int index, GroupType parquetType,
+            ParserContext context) {
+        super(parent, fieldName, index, parquetType, context);
+    }
+
+    @Override
+    public void start() {
+        tempStorage = context.enterCollection();
+        builder = context.getCollectionBuilder(DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE);
+    }
+
+    @Override
+    public void end() {
+        try {
+            builder.write(getParentDataOutput(), true);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+        addThisValueToParent();
+        context.exitCollection(tempStorage, builder);
+        tempStorage = null;
+        builder = null;
+    }
+
+    @Override
+    protected void addValue(IFieldValue value) {
+        try {
+            builder.addItem(tempStorage);
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    @Override
+    protected AtomicConverter createAtomicConverter(GroupType type, int index) {
+        return new AtomicConverter(this, index, context);
+    }
+
+    @Override
+    protected ArrayConverter createArrayConverter(GroupType type, int index) {
+        final GroupType arrayType = type.getType(index).asGroupType();
+        return new ArrayConverter(this, index, arrayType, context);
+    }
+
+    @Override
+    protected ObjectConverter createObjectConverter(GroupType type, int index) {
+        final GroupType objectType = type.getType(index).asGroupType();
+        return new ObjectConverter(this, index, objectType, context);
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AtomicConverter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AtomicConverter.java
new file mode 100644
index 0000000..7219bdd
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/AtomicConverter.java
@@ -0,0 +1,121 @@
+/*
+ * 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.input.record.reader.hdfs.parquet;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.external.input.stream.StandardUTF8ToModifiedUTF8DataOutput;
+import org.apache.asterix.external.parser.jackson.ParserContext;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.io.api.PrimitiveConverter;
+
+/**
+ * Currently, only JSON types are supported (string, number, boolean)
+ */
+class AtomicConverter extends PrimitiveConverter implements IFieldValue {
+    private final AbstractComplexConverter parent;
+    private final IValueReference fieldName;
+    private final int index;
+    private final ParserContext context;
+
+    public AtomicConverter(AbstractComplexConverter parent, int index, ParserContext context) {
+        this(parent, null, index, context);
+    }
+
+    public AtomicConverter(AbstractComplexConverter parent, IValueReference fieldName, int index,
+            ParserContext context) {
+        this.parent = parent;
+        this.fieldName = fieldName;
+        this.index = index;
+        this.context = context;
+    }
+
+    @Override
+    public void addBinary(Binary value) {
+        final DataOutput out = parent.getDataOutput();
+        final StandardUTF8ToModifiedUTF8DataOutput stringOut = context.getModifiedUTF8DataOutput();
+        stringOut.setDataOutput(out);
+        try {
+            out.writeByte(ATypeTag.STRING.serialize());
+            value.writeTo(stringOut);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+        parent.addValue(this);
+    }
+
+    @Override
+    public void addBoolean(boolean value) {
+        final DataOutput out = parent.getDataOutput();
+        try {
+            out.writeByte(ATypeTag.BOOLEAN.serialize());
+            out.writeBoolean(value);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+        parent.addValue(this);
+    }
+
+    @Override
+    public void addFloat(float value) {
+        addDouble(value);
+    }
+
+    @Override
+    public void addDouble(double value) {
+        final DataOutput out = parent.getDataOutput();
+        try {
+            out.writeByte(ATypeTag.DOUBLE.serialize());
+            out.writeDouble(value);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+        parent.addValue(this);
+    }
+
+    @Override
+    public void addInt(int value) {
+        addLong(value);
+    }
+
+    @Override
+    public void addLong(long value) {
+        final DataOutput out = parent.getDataOutput();
+        try {
+            out.writeByte(ATypeTag.BIGINT.serialize());
+            out.writeLong(value);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+        parent.addValue(this);
+    }
+
+    @Override
+    public IValueReference getFieldName() {
+        return fieldName;
+    }
+
+    @Override
+    public int getIndex() {
+        return index;
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/IFieldValue.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/IFieldValue.java
new file mode 100644
index 0000000..946b10c
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/IFieldValue.java
@@ -0,0 +1,34 @@
+/*
+ * 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.input.record.reader.hdfs.parquet;
+
+import org.apache.hyracks.data.std.api.IValueReference;
+
+/**
+ * This interface is intended to extend {@link org.apache.parquet.io.api.Converter} to get the field name or the
+ * index of associated with a value.
+ */
+interface IFieldValue {
+    IValueReference getFieldName();
+
+    /**
+     * @return the index of the value as appeared in the schema
+     */
+    int getIndex();
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/MapredParquetInputFormat.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/MapredParquetInputFormat.java
new file mode 100644
index 0000000..328e09d
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/MapredParquetInputFormat.java
@@ -0,0 +1,219 @@
+/*
+ * 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.input.record.reader.hdfs.parquet;
+
+import static java.util.Arrays.asList;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.parquet.hadoop.Footer;
+import org.apache.parquet.hadoop.ParquetInputFormat;
+import org.apache.parquet.hadoop.ParquetInputSplit;
+import org.apache.parquet.hadoop.ParquetRecordReader;
+
+/**
+ * For the Original implementation, see {@code DeprecatedParquetInputFormat}
+ * The original implementation has been modified to have {@code VoidPointable}
+ * instead of {@code org.apache.parquet.hadoop.mapred.Container}
+ * <p>
+ * AsterixDB currently support the older Hadoop API (@see org.apache.hadoop.mapred).
+ * The newer API (@see org.apache.hadoop.mapreduce) is not yet supported.
+ * Beware before upgrading Apache Parquet version.
+ */
+public class MapredParquetInputFormat extends org.apache.hadoop.mapred.FileInputFormat<Void, VoidPointable> {
+
+    protected ParquetInputFormat<ArrayBackedValueStorage> realInputFormat = new ParquetInputFormat<>();
+
+    @Override
+    public RecordReader<Void, VoidPointable> getRecordReader(InputSplit split, JobConf job, Reporter reporter)
+            throws IOException {
+        return new RecordReaderWrapper(split, job, reporter);
+    }
+
+    @Override
+    public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
+        if (isTaskSideMetaData(job)) {
+            return super.getSplits(job, numSplits);
+        }
+
+        List<Footer> footers = getFooters(job);
+        List<ParquetInputSplit> splits = realInputFormat.getSplits(job, footers);
+        if (splits == null) {
+            return null; //NOSONAR
+        }
+        InputSplit[] resultSplits = new InputSplit[splits.size()];
+        int i = 0;
+        for (ParquetInputSplit split : splits) {
+            resultSplits[i++] = new ParquetInputSplitWrapper(split);
+        }
+        return resultSplits;
+    }
+
+    public List<Footer> getFooters(JobConf job) throws IOException {
+        return realInputFormat.getFooters(job, asList(super.listStatus(job)));
+    }
+
+    private static class RecordReaderWrapper implements RecordReader<Void, VoidPointable> {
+
+        private final ParquetRecordReader<IValueReference> realReader;
+        private final long splitLen; // for getPos()
+
+        private final VoidPointable valueContainer;
+
+        private boolean firstRecord;
+        private boolean eof;
+
+        public RecordReaderWrapper(InputSplit oldSplit, JobConf oldJobConf, Reporter reporter) throws IOException {
+            splitLen = oldSplit.getLength();
+
+            try {
+                realReader = new ParquetRecordReader<>(
+                        ParquetInputFormat.<IValueReference> getReadSupportInstance(oldJobConf),
+                        ParquetInputFormat.getFilter(oldJobConf));
+
+                if (oldSplit instanceof ParquetInputSplitWrapper) {
+                    realReader.initialize(((ParquetInputSplitWrapper) oldSplit).realSplit, oldJobConf, reporter);
+                } else if (oldSplit instanceof FileSplit) {
+                    realReader.initialize((FileSplit) oldSplit, oldJobConf, reporter);
+                } else {
+                    throw new IllegalArgumentException(
+                            "Invalid split (not a FileSplit or ParquetInputSplitWrapper): " + oldSplit);
+                }
+                valueContainer = new VoidPointable();
+                firstRecord = false;
+                eof = false;
+                // read once to gain access to key and value objects
+                if (realReader.nextKeyValue()) {
+                    firstRecord = true;
+                    valueContainer.set(realReader.getCurrentValue());
+
+                } else {
+                    eof = true;
+                }
+            } catch (InterruptedException e) {
+                throw new IOException(e);
+            }
+        }
+
+        @Override
+        public void close() throws IOException {
+            realReader.close();
+        }
+
+        @Override
+        public Void createKey() {
+            return null;
+        }
+
+        @Override
+        public VoidPointable createValue() {
+            return valueContainer;
+        }
+
+        @Override
+        public long getPos() throws IOException {
+            return (long) (splitLen * getProgress());
+        }
+
+        @Override
+        public float getProgress() throws IOException {
+            try {
+                return realReader.getProgress();
+            } catch (InterruptedException e) {
+                throw new IOException(e);
+            }
+        }
+
+        @Override
+        public boolean next(Void key, VoidPointable value) throws IOException {
+            if (eof) {
+                return false;
+            }
+
+            if (firstRecord) { // key & value are already read.
+                firstRecord = false;
+                value.set(valueContainer);
+                return true;
+            }
+
+            try {
+                if (realReader.nextKeyValue()) {
+                    if (value != null) {
+                        value.set(realReader.getCurrentValue());
+                    }
+                    return true;
+                }
+            } catch (InterruptedException e) {
+                throw new IOException(e);
+            }
+
+            eof = true; // strictly not required, just for consistency
+            return false;
+        }
+    }
+
+    public static boolean isTaskSideMetaData(JobConf job) {
+        return job.getBoolean(ParquetInputFormat.TASK_SIDE_METADATA, true);
+    }
+
+    private static class ParquetInputSplitWrapper implements InputSplit {
+
+        ParquetInputSplit realSplit;
+
+        @SuppressWarnings("unused") // MapReduce instantiates this.
+        public ParquetInputSplitWrapper() {
+        }
+
+        public ParquetInputSplitWrapper(ParquetInputSplit realSplit) {
+            this.realSplit = realSplit;
+        }
+
+        @Override
+        public long getLength() throws IOException {
+            return realSplit.getLength();
+        }
+
+        @Override
+        public String[] getLocations() throws IOException {
+            return realSplit.getLocations();
+        }
+
+        @Override
+        public void readFields(DataInput in) throws IOException {
+            realSplit = new ParquetInputSplit();
+            realSplit.readFields(in);
+        }
+
+        @Override
+        public void write(DataOutput out) throws IOException {
+            realSplit.write(out);
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ObjectConverter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ObjectConverter.java
new file mode 100644
index 0000000..8736e7d
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ObjectConverter.java
@@ -0,0 +1,100 @@
+/*
+ * 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.input.record.reader.hdfs.parquet;
+
+import java.io.IOException;
+
+import org.apache.asterix.builders.IARecordBuilder;
+import org.apache.asterix.external.parser.jackson.ParserContext;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.schema.GroupType;
+
+class ObjectConverter extends AbstractComplexConverter {
+    private IARecordBuilder builder;
+
+    public ObjectConverter(AbstractComplexConverter parent, int index, GroupType parquetType, ParserContext context) {
+        super(parent, index, parquetType, context);
+    }
+
+    public ObjectConverter(AbstractComplexConverter parent, IValueReference fieldName, int index, GroupType parquetType,
+            ParserContext context) {
+        super(parent, fieldName, index, parquetType, context);
+    }
+
+    @Override
+    public void start() {
+        tempStorage = context.enterObject();
+        builder = context.getObjectBuilder(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+    }
+
+    @Override
+    public void end() {
+        try {
+            builder.write(getParentDataOutput(), true);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+        addThisValueToParent();
+        context.exitObject(tempStorage, null, builder);
+        tempStorage = null;
+        builder = null;
+    }
+
+    @Override
+    protected void addValue(IFieldValue value) {
+        try {
+            builder.addField(value.getFieldName(), getValue());
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    @Override
+    protected AtomicConverter createAtomicConverter(GroupType type, int index) {
+        try {
+            return new AtomicConverter(this, context.getSerializedFieldName(type.getFieldName(index)), index, context);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    @Override
+    protected ArrayConverter createArrayConverter(GroupType type, int index) {
+        try {
+            final IValueReference childFieldName = context.getSerializedFieldName(type.getFieldName(index));
+            final GroupType arrayType = type.getType(index).asGroupType();
+            return new ArrayConverter(this, childFieldName, index, arrayType, context);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    @Override
+    protected ObjectConverter createObjectConverter(GroupType type, int index) {
+        try {
+            final IValueReference childFieldName = context.getSerializedFieldName(type.getFieldName(index));
+            final GroupType objectType = type.getType(index).asGroupType();
+            return new ObjectConverter(this, childFieldName, index, objectType, context);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java
new file mode 100644
index 0000000..e208240
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetFileRecordReader.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.hdfs.parquet;
+
+import java.io.IOException;
+
+import org.apache.asterix.external.input.record.ValueReferenceRecord;
+import org.apache.asterix.external.input.record.reader.hdfs.AbstractHDFSRecordReader;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+/**
+ * Apache Parquet record reader.
+ * The reader returns records in ADM format.
+ */
+public class ParquetFileRecordReader<V extends IValueReference> extends AbstractHDFSRecordReader<Void, V> {
+
+    public ParquetFileRecordReader(boolean[] read, InputSplit[] inputSplits, String[] readSchedule, String nodeName,
+            JobConf conf) {
+        super(read, inputSplits, readSchedule, nodeName, new ValueReferenceRecord<>(), conf);
+    }
+
+    @Override
+    protected boolean onNextInputSplit() throws IOException {
+        return false;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    protected RecordReader<Void, V> getRecordReader(int splitIndex) throws IOException {
+        reader = (RecordReader<Void, V>) inputFormat.getRecordReader(inputSplits[splitIndex], conf, Reporter.NULL);
+        if (value == null) {
+            value = reader.createValue();
+        }
+        return reader;
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetReadSupport.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetReadSupport.java
new file mode 100644
index 0000000..814da3c
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/ParquetReadSupport.java
@@ -0,0 +1,112 @@
+/*
+ * 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.input.record.reader.hdfs.parquet;
+
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.parquet.hadoop.api.InitContext;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.io.api.GroupConverter;
+import org.apache.parquet.io.api.RecordMaterializer;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
+import org.apache.parquet.schema.Type;
+import org.apache.parquet.schema.Type.Repetition;
+import org.apache.parquet.schema.Types;
+import org.apache.parquet.schema.Types.GroupBuilder;
+import org.apache.parquet.schema.Types.MessageTypeBuilder;
+
+public class ParquetReadSupport extends ReadSupport<IValueReference> {
+    private static final PrimitiveType NULL = Types.optional(PrimitiveTypeName.BOOLEAN).named("NULL");
+
+    @Override
+    public ReadContext init(InitContext context) {
+        final String requestedSchemaString = context.getConfiguration().get(ExternalDataConstants.KEY_REQUESTED_FIELDS);
+        final MessageType requestedSchema = getRequestedSchema(requestedSchemaString, context.getFileSchema());
+        return new ReadContext(requestedSchema, Collections.emptyMap());
+    }
+
+    @Override
+    public RecordMaterializer<IValueReference> prepareForRead(Configuration configuration,
+            Map<String, String> keyValueMetaData, MessageType fileSchema, ReadContext readContext) {
+        return new ADMRecordMaterializer(readContext);
+    }
+
+    private static class ADMRecordMaterializer extends RecordMaterializer<IValueReference> {
+        private final RootConverter rootConverter;
+
+        public ADMRecordMaterializer(ReadContext readContext) {
+            rootConverter = new RootConverter(readContext.getRequestedSchema());
+        }
+
+        @Override
+        public IValueReference getCurrentRecord() {
+            return rootConverter.getRecord();
+        }
+
+        @Override
+        public GroupConverter getRootConverter() {
+            return rootConverter;
+        }
+
+    }
+
+    private static MessageType getRequestedSchema(String requestedSchemaString, MessageType fileSchema) {
+        if ("*".equals(requestedSchemaString)) {
+            return fileSchema;
+        }
+
+        final MessageTypeBuilder builder = Types.buildMessage();
+        final String[] paths = requestedSchemaString.split(",");
+        for (int i = 0; i < paths.length; i++) {
+            buildRequestedType(paths[i].trim().split("[.]"), builder, fileSchema, 0);
+        }
+
+        return builder.named("asterix");
+
+    }
+
+    private static void buildRequestedType(String[] fieldNames, GroupBuilder<?> builder, GroupType groupType,
+            int start) {
+        final String fieldName = fieldNames[start].trim();
+
+        Type type = getType(groupType, fieldName);
+        if (type != NULL && start < fieldNames.length - 1) {
+            final GroupBuilder<GroupType> innerFieldBuilder = Types.buildGroup(Repetition.OPTIONAL);
+            buildRequestedType(fieldNames, innerFieldBuilder, type.asGroupType(), start + 1);
+            builder.addField(innerFieldBuilder.named(fieldName));
+        } else {
+            builder.addField(type);
+        }
+    }
+
+    private static Type getType(GroupType groupType, String fieldName) {
+        if (groupType.containsField(fieldName)) {
+            return groupType.getType(fieldName);
+        }
+        return NULL;
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/RepeatedConverter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/RepeatedConverter.java
new file mode 100644
index 0000000..d46d84c
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/RepeatedConverter.java
@@ -0,0 +1,66 @@
+/*
+ * 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.input.record.reader.hdfs.parquet;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.external.parser.jackson.ParserContext;
+import org.apache.parquet.schema.GroupType;
+
+class RepeatedConverter extends AbstractComplexConverter {
+    public RepeatedConverter(AbstractComplexConverter parent, int index, GroupType parquetType, ParserContext context) {
+        super(parent, index, parquetType, context);
+    }
+
+    @Override
+    public void start() {
+        //NoOp
+    }
+
+    @Override
+    public void end() {
+        //NoOp
+    }
+
+    @Override
+    protected void addValue(IFieldValue value) {
+        parent.addValue(value);
+    }
+
+    @Override
+    protected AtomicConverter createAtomicConverter(GroupType type, int index) {
+        return new AtomicConverter(this, index, context);
+    }
+
+    @Override
+    protected ArrayConverter createArrayConverter(GroupType type, int index) {
+        final GroupType arrayType = type.getType(index).asGroupType();
+        return new ArrayConverter(this, index, arrayType, context);
+    }
+
+    @Override
+    protected ObjectConverter createObjectConverter(GroupType type, int index) {
+        return new ObjectConverter(this, index, type.getType(index).asGroupType(), context);
+    }
+
+    @Override
+    protected DataOutput getDataOutput() {
+        return getParentDataOutput();
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/RootConverter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/RootConverter.java
new file mode 100644
index 0000000..76f4342
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/parquet/RootConverter.java
@@ -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.
+ */
+package org.apache.asterix.external.input.record.reader.hdfs.parquet;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.external.parser.jackson.ParserContext;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.parquet.schema.GroupType;
+
+class RootConverter extends ObjectConverter {
+    private final ArrayBackedValueStorage rootBuffer;
+
+    public RootConverter(GroupType parquetType) {
+        super(null, -1, parquetType, new ParserContext(true));
+        this.rootBuffer = new ArrayBackedValueStorage();
+    }
+
+    @Override
+    protected DataOutput getParentDataOutput() {
+        rootBuffer.reset();
+        return rootBuffer.getDataOutput();
+    }
+
+    protected IValueReference getRecord() {
+        return rootBuffer;
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AsterixInputStreamReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AsterixInputStreamReader.java
index 4e963e4..d2499f8 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AsterixInputStreamReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AsterixInputStreamReader.java
@@ -32,17 +32,20 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class AsterixInputStreamReader extends Reader {
-    private AsterixInputStream in;
-    private byte[] bytes = new byte[ExternalDataConstants.DEFAULT_BUFFER_SIZE];
-    private ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
-    private CharBuffer charBuffer = CharBuffer.allocate(ExternalDataConstants.DEFAULT_BUFFER_SIZE);
-    private CharsetDecoder decoder;
-    private boolean done = false;
-    private boolean remaining = false;
+    private final AsterixInputStream in;
+    private final CharsetDecoder decoder;
+    private final byte[] bytes;
+    protected final ByteBuffer byteBuffer;
+    protected final CharBuffer charBuffer;
+    protected boolean done = false;
+    protected boolean remaining = false;
 
     public AsterixInputStreamReader(AsterixInputStream in) {
         this.in = in;
         this.decoder = StandardCharsets.UTF_8.newDecoder();
+        bytes = new byte[ExternalDataConstants.DEFAULT_BUFFER_SIZE];
+        byteBuffer = ByteBuffer.wrap(bytes);
+        charBuffer = CharBuffer.allocate(ExternalDataConstants.DEFAULT_BUFFER_SIZE);
         this.byteBuffer.flip();
     }
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/StandardUTF8ToModifiedUTF8DataOutput.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/StandardUTF8ToModifiedUTF8DataOutput.java
new file mode 100644
index 0000000..be0d286
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/StandardUTF8ToModifiedUTF8DataOutput.java
@@ -0,0 +1,158 @@
+/*
+ * 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.input.stream;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
+
+/**
+ * Writes modified UTF-8 string format to {@link StandardUTF8ToModifiedUTF8DataOutput#out}
+ * from standard UTF-8 string format.
+ */
+public class StandardUTF8ToModifiedUTF8DataOutput implements DataOutput {
+    private static final byte[] EMPTY = new byte[0];
+    private final AStringSerializerDeserializer stringSerDer;
+    private final ResettableUTF8InputStreamReader reader;
+    private final char[] inputBuffer;
+    private char[] appendBuffer;
+    private DataOutput out;
+
+    public StandardUTF8ToModifiedUTF8DataOutput(AStringSerializerDeserializer stringSerDer) {
+        this.stringSerDer = stringSerDer;
+        reader = new ResettableUTF8InputStreamReader(new ByteArrayAccessibleInputStream(EMPTY, 0, 0));
+        inputBuffer = new char[ExternalDataConstants.DEFAULT_BUFFER_SIZE];
+        appendBuffer = new char[ExternalDataConstants.DEFAULT_BUFFER_SIZE];
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void write(byte[] b) throws IOException {
+        write(b, 0, b.length);
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+        reader.prepareNextRead(b, off, len);
+        int numOfChars = reader.read(inputBuffer);
+        int length = 0;
+        while (numOfChars > 0) {
+            appendBuffer = append(inputBuffer, appendBuffer, length, numOfChars);
+            length += numOfChars;
+            numOfChars = reader.read(inputBuffer);
+        }
+        stringSerDer.serialize(appendBuffer, 0, length, out);
+    }
+
+    @Override
+    public void writeBoolean(boolean v) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void writeByte(int v) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void writeShort(int v) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void writeChar(int v) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void writeInt(int v) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void writeLong(long v) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void writeFloat(float v) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void writeDouble(double v) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void writeBytes(String s) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void writeChars(String s) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void writeUTF(String s) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    public void setDataOutput(DataOutput out) {
+        this.out = out;
+    }
+
+    private static char[] append(char[] src, char[] dest, int offset, int length) {
+        char[] destBuf = dest;
+        if (offset + length > dest.length) {
+            char[] newDestBuffer = new char[dest.length * 2];
+            System.arraycopy(destBuf, 0, newDestBuffer, 0, offset);
+            destBuf = newDestBuffer;
+        }
+        System.arraycopy(src, 0, destBuf, offset, length);
+        return destBuf;
+    }
+
+    private static class ResettableUTF8InputStreamReader extends AsterixInputStreamReader {
+        private final ByteArrayAccessibleInputStream inByte;
+
+        public ResettableUTF8InputStreamReader(ByteArrayAccessibleInputStream inByte) {
+            super(new BasicInputStream(inByte));
+            this.inByte = inByte;
+        }
+
+        //Rewind the reader after setting the byte array
+        public void prepareNextRead(byte[] b, int off, int len) {
+            inByte.setContent(b, off, len);
+            done = false;
+            remaining = false;
+            byteBuffer.flip();
+            charBuffer.flip();
+        }
+
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/NoOpDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/NoOpDataParser.java
new file mode 100644
index 0000000..8a42558
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/NoOpDataParser.java
@@ -0,0 +1,40 @@
+/*
+ * 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.parser;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.external.api.IRawRecord;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public class NoOpDataParser implements IRecordDataParser<IValueReference> {
+
+    @Override
+    public boolean parse(IRawRecord<? extends IValueReference> record, DataOutput out) throws HyracksDataException {
+        try {
+            out.write(record.getBytes(), 0, record.size());
+            return true;
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/NoOpDataParserFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/NoOpDataParserFactory.java
new file mode 100644
index 0000000..f4fca17
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/NoOpDataParserFactory.java
@@ -0,0 +1,71 @@
+/*
+ * 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.parser.factory;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.external.api.IRecordDataParser;
+import org.apache.asterix.external.api.IRecordDataParserFactory;
+import org.apache.asterix.external.parser.NoOpDataParser;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+public class NoOpDataParserFactory implements IRecordDataParserFactory<IValueReference> {
+    private static final long serialVersionUID = 5274870143009767516L;
+
+    private static final List<String> PARSER_FORMAT = Collections.singletonList(ExternalDataConstants.FORMAT_NOOP);
+
+    @Override
+    public void configure(Map<String, String> configuration) throws AlgebricksException {
+        //nothing
+    }
+
+    @Override
+    public void setRecordType(ARecordType recordType) throws AsterixException {
+        //it always return open type
+    }
+
+    @Override
+    public void setMetaType(ARecordType metaType) {
+        //no meta type
+    }
+
+    @Override
+    public List<String> getParserFormats() {
+        return PARSER_FORMAT;
+    }
+
+    @Override
+    public IRecordDataParser<IValueReference> createRecordParser(IHyracksTaskContext ctx) throws HyracksDataException {
+        return new NoOpDataParser();
+    }
+
+    @Override
+    public Class<?> getRecordClass() {
+        return IValueReference.class;
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/jackson/ParserContext.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/jackson/ParserContext.java
index 387ba7f..d0b79b1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/jackson/ParserContext.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/jackson/ParserContext.java
@@ -27,6 +27,8 @@
 import org.apache.asterix.builders.IAsterixListBuilder;
 import org.apache.asterix.builders.ListBuilderFactory;
 import org.apache.asterix.builders.RecordBuilderFactory;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.external.input.stream.StandardUTF8ToModifiedUTF8DataOutput;
 import org.apache.asterix.external.parser.AbstractNestedDataParser;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMutableString;
@@ -39,6 +41,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IMutableValueStorage;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.util.string.UTF8StringReader;
+import org.apache.hyracks.util.string.UTF8StringWriter;
 
 /**
  * A state class that helps parsers of class {@link AbstractNestedDataParser} to maintain
@@ -54,7 +58,7 @@
      * Parsing nested structure using temporary buffers is expensive.
      * Example:
      * {"a":{"b":{"c":{"d":5}}}}
-     *
+     * <p>
      * Scalar value 5 is written 4 times in tempBuffer("d") then tempBuffer("c") ... tempBuffer("a")
      */
     private final ObjectPool<IMutableValueStorage, ATypeTag> tempBufferPool;
@@ -63,8 +67,15 @@
     private final ISerializerDeserializer<AString> stringSerDe;
     private final AMutableString aString;
 
-    @SuppressWarnings("unchecked")
+    //For parquet
+    private final StandardUTF8ToModifiedUTF8DataOutput modifiedUTF8DataOutput;
+
     public ParserContext() {
+        this(false);
+    }
+
+    @SuppressWarnings("unchecked")
+    public ParserContext(boolean allocateModfiedUTF8Writer) {
         objectBuilderPool = new ObjectPool<>(new RecordBuilderFactory());
         arrayBuilderPool = new ObjectPool<>(new ListBuilderFactory(), ATypeTag.ARRAY);
         tempBufferPool = new ObjectPool<>(new AbvsBuilderFactory());
@@ -72,6 +83,11 @@
         serializedFieldNames = new LRUMap<>(SERIALIZED_FIELDNAME_MAP_MAX_SIZE);
         stringSerDe = SerializerDeserializerProvider.INSTANCE.getAStringSerializerDeserializer();
         aString = new AMutableString("");
+        modifiedUTF8DataOutput =
+                allocateModfiedUTF8Writer
+                        ? new StandardUTF8ToModifiedUTF8DataOutput(
+                                new AStringSerializerDeserializer(new UTF8StringWriter(), new UTF8StringReader()))
+                        : null;
     }
 
     public IMutableValueStorage enterObject() {
@@ -140,4 +156,8 @@
         arrayBuilderPool.recycle(builder);
     }
 
+    public StandardUTF8ToModifiedUTF8DataOutput getModifiedUTF8DataOutput() {
+        return modifiedUTF8DataOutput;
+    }
+
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index e20270c..dcd58de 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -117,9 +117,10 @@
     // a string representing the NULL value
     public static final String KEY_NULL_STR = "null";
     public static final String KEY_REDACT_WARNINGS = "redact-warnings";
+    public static final String KEY_REQUESTED_FIELDS = "requested-fields";
 
     /**
-     *  Keys for adapter name
+     * Keys for adapter name
      **/
     public static final String KEY_ADAPTER_NAME_TWITTER_PUSH = "twitter_push";
     public static final String KEY_ADAPTER_NAME_PUSH_TWITTER = "push_twitter";
@@ -139,6 +140,8 @@
     public static final String CLASS_NAME_TEXT_INPUT_FORMAT = "org.apache.hadoop.mapred.TextInputFormat";
     public static final String CLASS_NAME_SEQUENCE_INPUT_FORMAT = "org.apache.hadoop.mapred.SequenceFileInputFormat";
     public static final String CLASS_NAME_RC_INPUT_FORMAT = "org.apache.asterix.hivecompat.io.RCFileInputFormat";
+    public static final String CLASS_NAME_PARQUET_INPUT_FORMAT =
+            "org.apache.asterix.external.input.record.reader.hdfs.parquet.MapredParquetInputFormat";
     public static final String CLASS_NAME_HDFS_FILESYSTEM = "org.apache.hadoop.hdfs.DistributedFileSystem";
     /**
      * input formats aliases
@@ -146,6 +149,7 @@
     public static final String INPUT_FORMAT_TEXT = "text-input-format";
     public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
     public static final String INPUT_FORMAT_RC = "rc-input-format";
+    public static final String INPUT_FORMAT_PARQUET = "parquet-input-format";
     /**
      * Builtin streams
      */
@@ -179,10 +183,12 @@
     public static final String FORMAT_SEMISTRUCTURED = "semi-structured";
     public static final String FORMAT_LINE_SEPARATED = "line-separated";
     public static final String FORMAT_HDFS_WRITABLE = "hdfs-writable";
+    public static final String FORMAT_NOOP = "noop";
     public static final String FORMAT_KV = "kv";
     public static final String FORMAT_CSV = "csv";
     public static final String FORMAT_TSV = "tsv";
     public static final Set<String> ALL_FORMATS;
+
     static {
         Set<String> formats = new HashSet<>(13);
         formats.add(FORMAT_HIVE);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index 86eb5e0..617903d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -89,6 +89,7 @@
 public class ExternalDataUtils {
 
     private static final Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = new EnumMap<>(ATypeTag.class);
+
     static {
         valueParserFactoryMap.put(ATypeTag.INTEGER, IntegerParserFactory.INSTANCE);
         valueParserFactoryMap.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
@@ -400,13 +401,18 @@
      * Prepares the configuration of the external data and its adapter by filling the information required by
      * adapters and parsers.
      *
-     * @param adapterName adapter name
+     * @param adapterName   adapter name
      * @param configuration external data configuration
      */
     public static void prepare(String adapterName, Map<String, String> configuration) {
         if (!configuration.containsKey(ExternalDataConstants.KEY_READER)) {
             configuration.put(ExternalDataConstants.KEY_READER, adapterName);
         }
+        final String inputFormat = configuration.get(ExternalDataConstants.KEY_INPUT_FORMAT);
+        if (ExternalDataConstants.INPUT_FORMAT_PARQUET.equals(inputFormat)) {
+            //Parquet supports binary-to-binary conversion. No parsing is required
+            configuration.put(ExternalDataConstants.KEY_FORMAT, ExternalDataConstants.FORMAT_NOOP);
+        }
         if (!configuration.containsKey(ExternalDataConstants.KEY_PARSER)
                 && configuration.containsKey(ExternalDataConstants.KEY_FORMAT)) {
             configuration.put(ExternalDataConstants.KEY_PARSER, configuration.get(ExternalDataConstants.KEY_FORMAT));
@@ -523,7 +529,6 @@
      * Regex matches all the provided patterns against the provided path
      *
      * @param path path to check against
-     *
      * @return {@code true} if all patterns match, {@code false} otherwise
      */
     public static boolean matchPatterns(List<Matcher> matchers, String path) {
@@ -539,7 +544,6 @@
      * Converts the wildcard to proper regex
      *
      * @param pattern wildcard pattern to convert
-     *
      * @return regex expression
      */
     public static String patternToRegex(String pattern) {
@@ -682,6 +686,14 @@
         }
     }
 
+    public static boolean supportsPushdown(Map<String, String> properties) {
+        //Currently, only Apache Parquet format is supported
+        return ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT
+                .equals(properties.get(ExternalDataConstants.KEY_INPUT_FORMAT))
+                || ExternalDataConstants.INPUT_FORMAT_PARQUET
+                        .equals(properties.get(ExternalDataConstants.KEY_INPUT_FORMAT));
+    }
+
     public static class AwsS3 {
         private AwsS3() {
             throw new AssertionError("do not instantiate");
@@ -740,7 +752,6 @@
          * Validate external dataset properties
          *
          * @param configuration properties
-         *
          * @throws CompilationException Compilation exception
          */
         public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
@@ -827,7 +838,6 @@
          * Validate external dataset properties
          *
          * @param configuration properties
-         *
          * @throws CompilationException Compilation exception
          */
         public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
index 1b13ec5..126b86f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/HDFSUtils.java
@@ -33,6 +33,8 @@
 import org.apache.asterix.external.indexing.ExternalFile;
 import org.apache.asterix.external.indexing.IndexingScheduler;
 import org.apache.asterix.external.indexing.RecordId.RecordIdType;
+import org.apache.asterix.external.input.record.reader.hdfs.parquet.MapredParquetInputFormat;
+import org.apache.asterix.external.input.record.reader.hdfs.parquet.ParquetReadSupport;
 import org.apache.asterix.external.input.stream.HDFSInputStream;
 import org.apache.asterix.hivecompat.io.RCFileInputFormat;
 import org.apache.hadoop.fs.BlockLocation;
@@ -51,9 +53,13 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.hdfs.scheduler.Scheduler;
+import org.apache.parquet.hadoop.ParquetInputFormat;
 
 public class HDFSUtils {
 
+    private HDFSUtils() {
+    }
+
     public static Scheduler initializeHDFSScheduler(ICCServiceContext serviceCtx) throws HyracksDataException {
         ICCContext ccContext = serviceCtx.getCCContext();
         Scheduler scheduler = null;
@@ -167,6 +173,8 @@
                 return ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT;
             case ExternalDataConstants.INPUT_FORMAT_RC:
                 return ExternalDataConstants.CLASS_NAME_RC_INPUT_FORMAT;
+            case ExternalDataConstants.INPUT_FORMAT_PARQUET:
+                return ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT;
             default:
                 return inputFormatParameter;
         }
@@ -181,6 +189,8 @@
                 return SequenceFileInputFormat.class;
             case ExternalDataConstants.INPUT_FORMAT_RC:
                 return RCFileInputFormat.class;
+            case ExternalDataConstants.INPUT_FORMAT_PARQUET:
+                return MapredParquetInputFormat.class;
             default:
                 return Class.forName(inputFormatParameter);
         }
@@ -188,14 +198,18 @@
 
     public static JobConf configureHDFSJobConf(Map<String, String> configuration) {
         JobConf conf = new JobConf();
-
         String localShortCircuitSocketPath = configuration.get(ExternalDataConstants.KEY_LOCAL_SOCKET_PATH);
         String formatClassName = HDFSUtils.getInputFormatClassName(configuration);
-        conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI,
-                configuration.get(ExternalDataConstants.KEY_HDFS_URL).trim());
-        conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_CLASS, ExternalDataConstants.CLASS_NAME_HDFS_FILESYSTEM);
-        conf.setClassLoader(HDFSInputStream.class.getClassLoader());
+        final String url = configuration.get(ExternalDataConstants.KEY_HDFS_URL);
+
+        //Allow hdfs adapter to read from local-files. However, this only works in a single-node configuration.
+        if (url != null && url.trim().startsWith("hdfs")) {
+            conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_CLASS,
+                    ExternalDataConstants.CLASS_NAME_HDFS_FILESYSTEM);
+            conf.set(ExternalDataConstants.KEY_HADOOP_FILESYSTEM_URI, url);
+        }
         conf.set(ExternalDataConstants.KEY_HADOOP_INPUT_DIR, configuration.get(ExternalDataConstants.KEY_PATH).trim());
+        conf.setClassLoader(HDFSInputStream.class.getClassLoader());
         conf.set(ExternalDataConstants.KEY_HADOOP_INPUT_FORMAT, formatClassName);
 
         // Enable local short circuit reads if user supplied the parameters
@@ -203,6 +217,15 @@
             conf.set(ExternalDataConstants.KEY_HADOOP_SHORT_CIRCUIT, "true");
             conf.set(ExternalDataConstants.KEY_HADOOP_SOCKET_PATH, localShortCircuitSocketPath.trim());
         }
+
+        if (ExternalDataConstants.CLASS_NAME_PARQUET_INPUT_FORMAT.equals(formatClassName)) {
+            //Parquet configurations
+            conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, ParquetReadSupport.class.getName());
+            //Set the requested fields. Default * which means all fields
+            final String requestedFields = configuration.get(ExternalDataConstants.KEY_REQUESTED_FIELDS);
+            conf.set(ExternalDataConstants.KEY_REQUESTED_FIELDS, requestedFields != null ? requestedFields : "*");
+        }
+
         return conf;
     }
 
@@ -213,7 +236,6 @@
             ArrayList<String> locs = new ArrayList<>();
             Map<String, String[]> stores = appCtx.getMetadataProperties().getStores();
             for (String node : stores.keySet()) {
-
                 int numIODevices = clusterStateManager.getIODevices(node).length;
                 for (int k = 0; k < numIODevices; k++) {
                     locs.add(node);
diff --git a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IDataParserFactory b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IDataParserFactory
index 7ce2048..7c2daed 100644
--- a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IDataParserFactory
+++ b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IDataParserFactory
@@ -21,4 +21,5 @@
 org.apache.asterix.external.parser.factory.HiveDataParserFactory
 org.apache.asterix.external.parser.factory.RecordWithMetadataParserFactory
 org.apache.asterix.external.parser.factory.RSSParserFactory
-org.apache.asterix.external.parser.factory.TweetParserFactory
\ No newline at end of file
+org.apache.asterix.external.parser.factory.TweetParserFactory
+org.apache.asterix.external.parser.factory.NoOpDataParserFactory
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
index d74a3f3..454d3da 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
@@ -33,6 +33,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
 import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
@@ -162,6 +163,6 @@
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
             List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
             ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
-            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
-            throws AlgebricksException;
+            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
+            IProjectionInfo<?> projectionInfo) throws AlgebricksException;
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index d30e3b3..2ac8bf9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -18,7 +18,9 @@
  */
 package org.apache.asterix.metadata.declared;
 
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -40,6 +42,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -98,8 +101,8 @@
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
             List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
             ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
-            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
-            throws AlgebricksException {
+            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
+            IProjectionInfo<?> projectionInfo) throws AlgebricksException {
         switch (dataset.getDatasetType()) {
             case EXTERNAL:
                 if (tupleFilterFactory != null || outputLimit >= 0) {
@@ -112,9 +115,9 @@
                         externalDataset.getItemTypeDataverseName(), itemTypeName).getDatatype();
 
                 ExternalDatasetDetails edd = (ExternalDatasetDetails) externalDataset.getDatasetDetails();
-                ITypedAdapterFactory adapterFactory =
-                        metadataProvider.getConfiguredAdapterFactory(externalDataset, edd.getAdapter(),
-                                edd.getProperties(), (ARecordType) itemType, null, context.getWarningCollector());
+                Map<String, String> properties = addProjectionInfo(projectionInfo, edd.getProperties());
+                ITypedAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(externalDataset,
+                        edd.getAdapter(), properties, (ARecordType) itemType, null, context.getWarningCollector());
                 return metadataProvider.buildExternalDatasetDataScannerRuntime(jobSpec, itemType, adapterFactory);
             case INTERNAL:
                 DataSourceId id = getId();
@@ -134,6 +137,17 @@
         }
     }
 
+    private Map<String, String> addProjectionInfo(IProjectionInfo<?> projectionInfo, Map<String, String> properties) {
+        Map<String, String> propertiesCopy = properties;
+        if (projectionInfo != null) {
+            //properties could be cached and reused, so we make a copy per query
+            propertiesCopy = new HashMap<>(properties);
+            ExternalDataProjectionInfo fieldNamesInfo = (ExternalDataProjectionInfo) projectionInfo;
+            fieldNamesInfo.addToProperties(propertiesCopy);
+        }
+        return propertiesCopy;
+    }
+
     private int[] createFilterIndexes(List<LogicalVariable> filterVars, IOperatorSchema opSchema) {
         if (filterVars != null && !filterVars.isEmpty()) {
             final int size = filterVars.size();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ExternalDataProjectionInfo.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ExternalDataProjectionInfo.java
new file mode 100644
index 0000000..fcbf522
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ExternalDataProjectionInfo.java
@@ -0,0 +1,98 @@
+/*
+ * 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.metadata.declared;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+
+public class ExternalDataProjectionInfo implements IProjectionInfo<List<String>> {
+    private final List<List<String>> projectedFieldNames;
+
+    public ExternalDataProjectionInfo() {
+        projectedFieldNames = new ArrayList<>();
+    }
+
+    private ExternalDataProjectionInfo(List<List<String>> projectedFieldNames) {
+        this.projectedFieldNames = new ArrayList<>();
+        for (List<String> path : projectedFieldNames) {
+            List<String> newPath = new ArrayList<>(path);
+            this.projectedFieldNames.add(newPath);
+        }
+    }
+
+    @Override
+    public List<List<String>> getProjectionInfo() {
+        return projectedFieldNames;
+    }
+
+    @Override
+    public IProjectionInfo<List<String>> createCopy() {
+        return new ExternalDataProjectionInfo(projectedFieldNames);
+    }
+
+    @Override
+    public boolean equals(Object other) {
+        if (!(other instanceof ExternalDataProjectionInfo)) {
+            return false;
+        }
+        List<List<String>> otherProjectedFieldNames = ((ExternalDataProjectionInfo) other).projectedFieldNames;
+        return projectedFieldNames.size() == otherProjectedFieldNames.size()
+                && VariableUtilities.varListEqualUnordered(projectedFieldNames, otherProjectedFieldNames);
+    }
+
+    public String toString() {
+        if (projectedFieldNames.isEmpty()) {
+            return "";
+        }
+        final StringBuilder fieldNamesBuilder = new StringBuilder();
+        append(fieldNamesBuilder);
+        return fieldNamesBuilder.toString();
+    }
+
+    /**
+     * Append projected field names to the external dataset properties
+     */
+    public void addToProperties(Map<String, String> properties) {
+        final String pushedFieldNames = toString();
+        if (!pushedFieldNames.isEmpty()) {
+            properties.put(ExternalDataConstants.KEY_REQUESTED_FIELDS, toString());
+        }
+    }
+
+    private void append(StringBuilder builder) {
+        appendFieldNames(projectedFieldNames.get(0), builder);
+        for (int i = 1; i < projectedFieldNames.size(); i++) {
+            builder.append(", ");
+            appendFieldNames(projectedFieldNames.get(i), builder);
+        }
+    }
+
+    private void appendFieldNames(List<String> fieldNames, StringBuilder builder) {
+        builder.append(fieldNames.get(0));
+        for (int i = 1; i < fieldNames.size(); i++) {
+            builder.append('.').append(fieldNames.get(i));
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
index 280a4fd..dc7b19b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
@@ -42,6 +42,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -162,8 +163,8 @@
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
             List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
             ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
-            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
-            throws AlgebricksException {
+            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
+            IProjectionInfo<?> projectionInfo) throws AlgebricksException {
         try {
             if (tupleFilterFactory != null || outputLimit >= 0) {
                 throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
index 1303b21..d415f73 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
@@ -41,6 +41,7 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
@@ -83,8 +84,8 @@
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
             List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
             ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
-            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
-            throws AlgebricksException {
+            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
+            IProjectionInfo<?> projectionInfo) throws AlgebricksException {
         if (tupleFilterFactory != null || outputLimit >= 0) {
             throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
                     "tuple filter and limit are not supported by FunctionDataSource");
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
index 70ff5de..f766da3 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
@@ -133,8 +134,8 @@
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
             List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
             ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
-            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
-            throws AlgebricksException {
+            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
+            IProjectionInfo<?> projectionInfo) throws AlgebricksException {
         if (tupleFilterFactory != null || outputLimit >= 0) {
             throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
                     "tuple filter and limit are not supported by LoadableDataSource");
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index d84cade..dd8b3b4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -117,6 +117,7 @@
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -462,10 +463,10 @@
             List<LogicalVariable> projectVariables, boolean projectPushed, List<LogicalVariable> minFilterVars,
             List<LogicalVariable> maxFilterVars, ITupleFilterFactory tupleFilterFactory, long outputLimit,
             IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec,
-            Object implConfig) throws AlgebricksException {
+            Object implConfig, IProjectionInfo<?> projectionInfo) throws AlgebricksException {
         return ((DataSource) dataSource).buildDatasourceScanRuntime(this, dataSource, scanVariables, projectVariables,
                 projectPushed, minFilterVars, maxFilterVars, tupleFilterFactory, outputLimit, opSchema, typeEnv,
-                context, jobSpec, implConfig);
+                context, jobSpec, implConfig, projectionInfo);
     }
 
     protected Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildLoadableDatasetScan(
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/projectedfieldnames/TestFieldNamesEquals.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/projectedfieldnames/TestFieldNamesEquals.java
new file mode 100644
index 0000000..d19a8bb
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/projectedfieldnames/TestFieldNamesEquals.java
@@ -0,0 +1,98 @@
+/*
+ * 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.metadata.projectedfieldnames;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.asterix.metadata.declared.ExternalDataProjectionInfo;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestFieldNamesEquals {
+
+    @Test
+    public void testEqual() {
+        ExternalDataProjectionInfo p1 = new ExternalDataProjectionInfo();
+        ExternalDataProjectionInfo p2 = new ExternalDataProjectionInfo();
+        setFieldNames(p1, new String[] { "a.b.c", "d.e.f", "g.h.i" });
+        setFieldNames(p2, new String[] { "a.b.c", "d.e.f", "g.h.i" });
+        Assert.assertEquals(p1, p2);
+    }
+
+    @Test
+    public void testReversed() {
+        ExternalDataProjectionInfo p1 = new ExternalDataProjectionInfo();
+        ExternalDataProjectionInfo p2 = new ExternalDataProjectionInfo();
+        setFieldNames(p1, new String[] { "a.b.c", "d.e.f", "g.h.i" });
+        setFieldNames(p2, new String[] { "g.h.i", "d.e.f", "a.b.c" });
+        Assert.assertEquals(p1, p2);
+    }
+
+    @Test
+    public void testDifferentPermutations() {
+        ExternalDataProjectionInfo p1 = new ExternalDataProjectionInfo();
+        ExternalDataProjectionInfo p2 = new ExternalDataProjectionInfo();
+        setFieldNames(p1, new String[] { "a.b.c", "d.e.f", "g.h.i" });
+        setFieldNames(p2, new String[] { "d.e.f", "g.h.i", "a.b.c" });
+        Assert.assertEquals(p1, p2);
+
+        p1 = new ExternalDataProjectionInfo();
+        p2 = new ExternalDataProjectionInfo();
+        setFieldNames(p1, new String[] { "a.b.c", "d.e.f", "g.h.i" });
+        setFieldNames(p2, new String[] { "g.h.i", "d.e.f", "a.b.c" });
+        Assert.assertEquals(p1, p2);
+
+        p1 = new ExternalDataProjectionInfo();
+        p2 = new ExternalDataProjectionInfo();
+        setFieldNames(p1, new String[] { "a.b.c", "d.e.f", "g.h.i" });
+        setFieldNames(p2, new String[] { "g.h.i", "a.b.c", "d.e.f" });
+        Assert.assertEquals(p1, p2);
+    }
+
+    @Test
+    public void testDifferentLengths() {
+        ExternalDataProjectionInfo p1 = new ExternalDataProjectionInfo();
+        ExternalDataProjectionInfo p2 = new ExternalDataProjectionInfo();
+        setFieldNames(p1, new String[] { "a.b.c", "d.e.f" });
+        setFieldNames(p2, new String[] { "d.e.f", "a.b.c", "g" });
+        Assert.assertNotEquals(p1, p2);
+    }
+
+    @Test
+    public void testEqualSubPath() {
+        ExternalDataProjectionInfo p1 = new ExternalDataProjectionInfo();
+        ExternalDataProjectionInfo p2 = new ExternalDataProjectionInfo();
+        setFieldNames(p1, new String[] { "a.b.c", "d.e.f.g" });
+        setFieldNames(p2, new String[] { "d.e.f", "a.b.c" });
+        Assert.assertNotEquals(p1, p2);
+    }
+
+    private static void setFieldNames(ExternalDataProjectionInfo p, String[] fieldNames) {
+        List<List<String>> fieldNamesList = p.getProjectionInfo();
+        for (String fnString : fieldNames) {
+            List<String> fnList = new ArrayList<>();
+            String[] fn = fnString.split("[.]");
+            Collections.addAll(fnList, fn);
+            fieldNamesList.add(fnList);
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index 8c07792..7a3206c 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -739,5 +739,10 @@
       <groupId>org.apache.logging.log4j</groupId>
       <artifactId>log4j-1.2-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.kitesdk</groupId>
+      <artifactId>kite-data-core</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index d69d62b..a6d4e7f 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -86,6 +86,7 @@
     <log4j.version>2.13.3</log4j.version>
     <awsjavasdk.version>2.10.83</awsjavasdk.version>
     <azurejavasdk.version>12.6.0</azurejavasdk.version>
+    <parquet.version>1.8.2</parquet.version>
 
     <implementation.title>Apache AsterixDB - ${project.name}</implementation.title>
     <implementation.url>https://asterixdb.apache.org/</implementation.url>
@@ -486,16 +487,16 @@
                 </pluginExecution>
                 <pluginExecution>
                   <pluginExecutionFilter>
-                   <groupId>net.revelc.code.formatter</groupId>
-                   <artifactId>formatter-maven-plugin</artifactId>
-                   <versionRange>[2.0.1,)</versionRange>
-                   <goals>
-                       <goal>format</goal>
-                   </goals>
-                   </pluginExecutionFilter>
-                   <action>
-                       <ignore></ignore>
-                   </action>
+                    <groupId>net.revelc.code.formatter</groupId>
+                    <artifactId>formatter-maven-plugin</artifactId>
+                    <versionRange>[2.0.1,)</versionRange>
+                    <goals>
+                      <goal>format</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore></ignore>
+                  </action>
                 </pluginExecution>
               </pluginExecutions>
             </lifecycleMappingMetadata>
@@ -1550,7 +1551,7 @@
       <dependency>
         <groupId>org.apache.commons</groupId>
         <artifactId>commons-csv</artifactId>
-	      <version>1.8</version>
+        <version>1.8</version>
       </dependency>
       <dependency>
         <groupId>org.testcontainers</groupId>
@@ -1572,6 +1573,21 @@
         <artifactId>msgpack-core</artifactId>
         <version>0.8.20</version>
       </dependency>
+      <dependency>
+        <groupId>org.apache.parquet</groupId>
+        <artifactId>parquet-column</artifactId>
+        <version>${parquet.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.parquet</groupId>
+        <artifactId>parquet-hadoop</artifactId>
+        <version>${parquet.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.kitesdk</groupId>
+        <artifactId>kite-data-core</artifactId>
+        <version>1.1.0</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index 7aa3640..717eac7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -51,8 +51,8 @@
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
             List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
             ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
-            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
-            throws AlgebricksException;
+            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig,
+            IProjectionInfo<?> projectionInfo) throws AlgebricksException;
 
     public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
             int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc)
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionInfo.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionInfo.java
new file mode 100644
index 0000000..9de591e
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IProjectionInfo.java
@@ -0,0 +1,37 @@
+/*
+ * 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.hyracks.algebricks.core.algebra.metadata;
+
+import java.util.List;
+
+/**
+ * Generic interface to include the projection information for
+ * {@link org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator}
+ */
+public interface IProjectionInfo<T> {
+    /**
+     * @return list of projected values' information
+     */
+    List<T> getProjectionInfo();
+
+    /**
+     * @return a copy of the {@link IProjectionInfo}
+     */
+    IProjectionInfo<T> createCopy();
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java
index c163c9f..64e41f5 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractScanOperator.java
@@ -46,7 +46,7 @@
 
     @Override
     public void recomputeSchema() {
-        schema = new ArrayList<LogicalVariable>();
+        schema = new ArrayList<>();
         schema.addAll(inputs.get(0).getValue().getSchema());
         schema.addAll(variables);
     }
@@ -67,5 +67,4 @@
             }
         };
     }
-
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
index 6f11dc1..bb18014 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
@@ -49,16 +50,19 @@
     // the maximum of number of results output by this operator
     private long outputLimit = -1;
 
+    private IProjectionInfo<?> projectionInfo;
+
     public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource) {
-        this(variables, dataSource, null, -1);
+        this(variables, dataSource, null, -1, null);
     }
 
     public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource,
-            Mutable<ILogicalExpression> selectCondition, long outputLimit) {
+            Mutable<ILogicalExpression> selectCondition, long outputLimit, IProjectionInfo projectionInfo) {
         super(variables, dataSource);
-        projectVars = new ArrayList<LogicalVariable>();
+        projectVars = new ArrayList<>();
         this.selectCondition = selectCondition;
         this.outputLimit = outputLimit;
+        this.projectionInfo = projectionInfo;
     }
 
     @Override
@@ -162,4 +166,12 @@
     public void setOutputLimit(long outputLimit) {
         this.outputLimit = outputLimit;
     }
+
+    public void setProjectionInfo(IProjectionInfo<?> projectionInfo) {
+        this.projectionInfo = projectionInfo;
+    }
+
+    public IProjectionInfo<?> getProjectionInfo() {
+        return projectionInfo;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index e50ea65..5dfdbbd 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -461,7 +461,9 @@
         }
         DataSourceScanOperator argScan = (DataSourceScanOperator) arg;
         boolean isomorphic = op.getDataSource().getId().equals(argScan.getDataSource().getId())
-                && op.getOutputLimit() == argScan.getOutputLimit();
+                && op.getOutputLimit() == argScan.getOutputLimit()
+                && Objects.equals(op.getProjectionInfo(), argScan.getProjectionInfo());
+
         if (!isomorphic) {
             return Boolean.FALSE;
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index 2cdcaa2..3a4010e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.IVariableContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
@@ -327,8 +328,9 @@
             throws AlgebricksException {
         Mutable<ILogicalExpression> newSelectCondition = op.getSelectCondition() != null
                 ? exprDeepCopyVisitor.deepCopyExpressionReference(op.getSelectCondition()) : null;
+        IProjectionInfo<?> projectionInfo = op.getProjectionInfo() != null ? op.getProjectionInfo().createCopy() : null;
         DataSourceScanOperator opCopy = new DataSourceScanOperator(deepCopyVariableList(op.getVariables()),
-                op.getDataSource(), newSelectCondition, op.getOutputLimit());
+                op.getDataSource(), newSelectCondition, op.getOutputLimit(), projectionInfo);
         deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
         return opCopy;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index e25c8c4..46999b0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -32,6 +32,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
@@ -255,9 +256,10 @@
         newInputList.addAll(op.getVariables());
         Mutable<ILogicalExpression> newSelectCondition =
                 op.getSelectCondition() != null ? deepCopyExpressionRef(op.getSelectCondition()) : null;
-        DataSourceScanOperator newOp =
-                new DataSourceScanOperator(newInputList, op.getDataSource(), newSelectCondition, op.getOutputLimit());
-        return newOp;
+        IProjectionInfo<?> projectionInfo = op.getProjectionInfo() != null ? op.getProjectionInfo().createCopy() : null;
+
+        return new DataSourceScanOperator(newInputList, op.getDataSource(), newSelectCondition, op.getOutputLimit(),
+                projectionInfo);
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
index 3ddb233..48990d2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
@@ -112,13 +112,14 @@
 
         ITupleFilterFactory tupleFilterFactory = null;
         if (scan.getSelectCondition() != null) {
-            tupleFilterFactory = context.getMetadataProvider().createTupleFilterFactory(
-                    new IOperatorSchema[] { opSchema }, typeEnv, scan.getSelectCondition().getValue(), context);
+            tupleFilterFactory = mp.createTupleFilterFactory(new IOperatorSchema[] { opSchema }, typeEnv,
+                    scan.getSelectCondition().getValue(), context);
         }
 
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = mp.getScannerRuntime(dataSource, vars, projectVars,
-                scan.isProjectPushed(), scan.getMinFilterVars(), scan.getMaxFilterVars(), tupleFilterFactory,
-                scan.getOutputLimit(), opSchema, typeEnv, context, builder.getJobSpec(), implConfig);
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p =
+                mp.getScannerRuntime(dataSource, vars, projectVars, scan.isProjectPushed(), scan.getMinFilterVars(),
+                        scan.getMaxFilterVars(), tupleFilterFactory, scan.getOutputLimit(), opSchema, typeEnv, context,
+                        builder.getJobSpec(), implConfig, scan.getProjectionInfo());
         IOperatorDescriptor opDesc = p.first;
         opDesc.setSourceLocation(scan.getSourceLocation());
         builder.contributeHyracksOperator(scan, opDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 1d560ce..bc8e024 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -30,6 +30,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
@@ -343,6 +344,7 @@
         appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars());
         appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
         appendLimitInformation(plan, op.getOutputLimit());
+        appendProjectInformation(plan, op.getProjectionInfo());
         return null;
     }
 
@@ -373,6 +375,15 @@
         }
     }
 
+    private void appendProjectInformation(AlgebricksStringBuilderWriter plan, IProjectionInfo<?> projectionInfo) {
+        final String projectedFields = projectionInfo == null ? "" : projectionInfo.toString();
+        if (!projectedFields.isEmpty()) {
+            plan.append(" project (");
+            plan.append(projectedFields);
+            plan.append(")");
+        }
+    }
+
     @Override
     public Void visitLimitOperator(LimitOperator op, Integer indent) throws AlgebricksException {
         addIndent(indent).append("limit " + op.getMaxObjects().getValue().accept(exprVisitor, indent));
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index 999a818..ac559d2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
@@ -499,6 +500,7 @@
             }
             writeFilterInformation(op.getMinFilterVars(), op.getMaxFilterVars());
             writeSelectLimitInformation(op.getSelectCondition(), op.getOutputLimit(), indent);
+            writeProjectInformation(op.getProjectionInfo());
             return null;
         } catch (IOException e) {
             throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
@@ -811,6 +813,13 @@
         }
     }
 
+    private void writeProjectInformation(IProjectionInfo<?> projectionInfo) throws IOException {
+        final String projectedFields = projectionInfo == null ? "" : projectionInfo.toString();
+        if (!projectedFields.isEmpty()) {
+            jsonGenerator.writeStringField("project", projectedFields);
+        }
+    }
+
     private void writeVariablesAndExpressions(List<LogicalVariable> variables,
             List<Mutable<ILogicalExpression>> expressions, Void indent) throws IOException, AlgebricksException {
         if (!variables.isEmpty()) {
@@ -847,6 +856,7 @@
     }
 
     /////////////// string fields ///////////////
+
     /** Writes "fieldName": "expr" */
     private void writeStringFieldExpression(String fieldName, Mutable<ILogicalExpression> expression, Void indent)
             throws AlgebricksException, IOException {
@@ -854,6 +864,7 @@
     }
 
     /////////////// array fields ///////////////
+
     /** Writes "fieldName": [ "var1", "var2", ... ] */
     private void writeArrayFieldOfVariables(String fieldName, List<LogicalVariable> variables) throws IOException {
         jsonGenerator.writeArrayFieldStart(fieldName);
@@ -925,6 +936,7 @@
     }
 
     /////////////// object fields ///////////////
+
     /** Writes "fieldName" : { "expressions": [ "expr1", "expr2", ...] } */
     private void writeObjectFieldWithExpressions(String fieldName, List<Mutable<ILogicalExpression>> exprs, Void indent)
             throws IOException, AlgebricksException {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index 33e27ac..566c11d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -25,8 +25,9 @@
 
     public static final String ALGEBRICKS_LOGGER_NAME = "org.apache.hyracks.algebricks";
     public static final Logger ALGEBRICKS_LOGGER = LogManager.getLogger(ALGEBRICKS_LOGGER_NAME);
-    public static final int SORT_SAMPLES = 100;
-    public static final boolean SORT_PARALLEL = true;
+    public static final int SORT_SAMPLES_DEFAULT = 100;
+    public static final boolean SORT_PARALLEL_DEFAULT = true;
     public static final boolean INDEX_ONLY_DEFAULT = true;
     public static final boolean SANITYCHECK_DEFAULT = false;
+    public static final boolean EXTERNAL_FIELD_PUSHDOWN_DEFAULT = false;
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 4f6fee0..e725cce 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -41,6 +41,7 @@
     private static final String SORT_SAMPLES = "SORT_SAMPLES";
     private static final String INDEX_ONLY = "INDEX_ONLY";
     private static final String SANITY_CHECK = "SANITY_CHECK";
+    private static final String EXTERNAL_FIELD_PUSHDOWN = "EXTERNAL_FIELD_PUSHDOWN";
 
     private Properties properties = new Properties();
 
@@ -159,7 +160,7 @@
     }
 
     public boolean getSortParallel() {
-        return getBoolean(SORT_PARALLEL, AlgebricksConfig.SORT_PARALLEL);
+        return getBoolean(SORT_PARALLEL, AlgebricksConfig.SORT_PARALLEL_DEFAULT);
     }
 
     public void setSortParallel(boolean sortParallel) {
@@ -167,7 +168,7 @@
     }
 
     public int getSortSamples() {
-        return getInt(SORT_SAMPLES, AlgebricksConfig.SORT_SAMPLES);
+        return getInt(SORT_SAMPLES, AlgebricksConfig.SORT_SAMPLES_DEFAULT);
     }
 
     public void setSortSamples(int sortSamples) {
@@ -190,6 +191,14 @@
         return getBoolean(SANITY_CHECK, AlgebricksConfig.SANITYCHECK_DEFAULT);
     }
 
+    public boolean isExternalFieldPushdown() {
+        return getBoolean(EXTERNAL_FIELD_PUSHDOWN, AlgebricksConfig.EXTERNAL_FIELD_PUSHDOWN_DEFAULT);
+    }
+
+    public void setExternalFieldPushdown(boolean externalFieldPushDown) {
+        setBoolean(EXTERNAL_FIELD_PUSHDOWN, externalFieldPushDown);
+    }
+
     private void setInt(String property, int value) {
         properties.setProperty(property, Integer.toString(value));
     }