Merge branch 'release-0.9.9'

Change-Id: I603ae267efd137d4e9f3491be2a6bdcb2179acdc
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 7dd0217..0949478 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -361,7 +361,7 @@
         return translate(expr, outputDatasetName, (ICompiledDmlStatement) stmt, null, resultMetadata);
     }
 
-    private ILogicalPlan translateCopyTo(Query expr, CompiledStatements.ICompiledStatement stmt,
+    public ILogicalPlan translateCopyTo(Query expr, CompiledStatements.ICompiledStatement stmt,
             IResultMetadata resultMetadata) throws AlgebricksException {
         CompiledStatements.CompiledCopyToStatement copyTo = (CompiledStatements.CompiledCopyToStatement) stmt;
         MutableObject<ILogicalOperator> base = new MutableObject<>(new EmptyTupleSourceOperator());
@@ -423,8 +423,7 @@
         // astPathExpressions has at least one expression see CopyToStatement constructor
         List<Expression> astPathExpressions = copyTo.getPathExpressions();
         ILogicalExpression fullPathExpr = null;
-        WriteDataSink writeDataSink;
-        String separator = String.valueOf(ExternalWriterProvider.getSeparator(copyTo.getAdapter()));
+        String separator = getExternalWriterSeparator(copyTo.getAdapter());
         List<Mutable<ILogicalExpression>> pathExprs = new ArrayList<>(astPathExpressions.size());
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> pathExprPair;
         for (int i = 0; i < astPathExpressions.size(); i++) {
@@ -453,11 +452,25 @@
             fullPathExpr = concat;
         }
 
+        // Handle key
+        boolean autogenerated = copyTo.isAutogenerated();
+        List<Expression> astKeyExpressions = copyTo.getKeyExpressions();
+        List<Mutable<ILogicalExpression>> keyExpressionRefs = new ArrayList<>(astKeyExpressions.size());
+        for (int i = 0; i < copyTo.getKeyExpressions().size(); i++) {
+            Expression expression = astKeyExpressions.get(i);
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> expPair = langExprToAlgExpression(expression, topOpRef);
+            keyExpressionRefs.add(new MutableObject<>(expPair.first));
+            Pair<Mutable<ILogicalExpression>, Mutable<ILogicalOperator>> wrappedPair =
+                    wrapInAssign(context.newVar(), expPair.first, expPair.second);
+            topOpRef = wrappedPair.second;
+        }
+
         // Write adapter configuration
-        writeDataSink = new WriteDataSink(copyTo.getAdapter(), copyTo.getProperties());
+        WriteDataSink writeDataSink = new WriteDataSink(copyTo.getAdapter(), copyTo.getProperties());
+
         // writeOperator
         WriteOperator writeOperator = new WriteOperator(sourceExprRef, new MutableObject<>(fullPathExpr),
-                partitionExpressionRefs, orderExprListOut, writeDataSink);
+                partitionExpressionRefs, orderExprListOut, keyExpressionRefs, autogenerated, writeDataSink);
         writeOperator.getInputs().add(topOpRef);
 
         // We need DistributeResultOperator to ensure all warnings to be delivered to the user
@@ -470,6 +483,10 @@
         return new ALogicalPlanImpl(globalPlanRoots);
     }
 
+    protected String getExternalWriterSeparator(String adapter) {
+        return String.valueOf(ExternalWriterProvider.getSeparator(adapter));
+    }
+
     public ILogicalPlan translate(Query expr, String outputDatasetName, ICompiledDmlStatement stmt,
             ILogicalOperator baseOp, IResultMetadata resultMetadata) throws AlgebricksException {
         MutableObject<ILogicalOperator> base = new MutableObject<>(new EmptyTupleSourceOperator());
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index f20a0319..6424280 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -1211,6 +1211,7 @@
         ExternalDetailsDecl externalDetails = (ExternalDetailsDecl) dd.getDatasetDetailsDecl();
         Map<String, String> properties = externalDetails.getProperties();
         ExternalDataUtils.validateParquetTypeAndConfiguration(properties, (ARecordType) itemType.getDatatype());
+        ExternalDataUtils.validateAvroTypeAndConfiguration(properties, (ARecordType) itemType.getDatatype());
         return properties;
     }
 
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java
index 90f46ad..55a515c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/ExternalDatasetTestUtils.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.test.external_dataset;
 
+import static org.apache.asterix.test.external_dataset.avro.AvroFileConverterUtil.AVRO_GEN_BASEDIR;
 import static org.apache.asterix.test.external_dataset.aws.AwsS3ExternalDatasetTest.BOM_FILE_CONTAINER;
 import static org.apache.asterix.test.external_dataset.aws.AwsS3ExternalDatasetTest.DYNAMIC_PREFIX_AT_START_CONTAINER;
 import static org.apache.asterix.test.external_dataset.aws.AwsS3ExternalDatasetTest.FIXED_DATA_CONTAINER;
@@ -33,6 +34,7 @@
 import java.nio.file.Paths;
 import java.util.Collection;
 
+import org.apache.asterix.test.external_dataset.avro.AvroFileConverterUtil;
 import org.apache.asterix.test.external_dataset.parquet.BinaryFileConverterUtil;
 import org.apache.asterix.testframework.context.TestCaseContext;
 import org.apache.commons.io.FilenameUtils;
@@ -51,6 +53,7 @@
     private static final FilenameFilter JSON_FILTER = ((dir, name) -> name.endsWith(".json"));
     private static final FilenameFilter CSV_FILTER = ((dir, name) -> name.endsWith(".csv"));
     private static final FilenameFilter PARQUET_FILTER = ((dir, name) -> name.endsWith(".parquet"));
+    private static final FilenameFilter AVRO_FILTER = ((dir, name) -> name.endsWith(".avro"));
 
     // Base directory paths for data files
     private static String JSON_DATA_PATH;
@@ -64,6 +67,7 @@
     public static final String TSV_DEFINITION = "tsv-data/reviews/";
     public static final String MIXED_DEFINITION = "mixed-data/reviews/";
     public static final String PARQUET_DEFINITION = "parquet-data/reviews/";
+    public static final String AVRO_DEFINITION = "avro-data/reviews/";
 
     // This is used for a test to generate over 1000 number of files
     public static final String OVER_1000_OBJECTS_PATH = "over-1000-objects";
@@ -101,6 +105,13 @@
         BinaryFileConverterUtil.convertToParquet(basePath, parquetRawJsonDir, BINARY_GEN_BASEDIR);
     }
 
+    public static void createAvroFiles(String avroRawJsonDir) throws IOException {
+        File basePath = new File(".");
+        // cleaning directory
+        BinaryFileConverterUtil.cleanBinaryDirectory(basePath, AVRO_GEN_BASEDIR);
+        AvroFileConverterUtil.convertToAvro(basePath, avroRawJsonDir, AVRO_GEN_BASEDIR);
+    }
+
     /**
      * Generate binary files (e.g., parquet files)
      */
@@ -114,6 +125,14 @@
                 JSON_FILTER, startIndex);
     }
 
+    public static void createAvroFilesRecursively(String dataToConvertDirPath) throws IOException {
+        //base path
+        File basePath = new File(".");
+        int startIndex = dataToConvertDirPath.indexOf("/external-filter");
+        AvroFileConverterUtil.convertToAvroRecursively(basePath, dataToConvertDirPath, AVRO_GEN_BASEDIR, JSON_FILTER,
+                startIndex);
+    }
+
     public static void setDataPaths(String jsonDataPath, String csvDataPath, String tsvDataPath) {
         JSON_DATA_PATH = jsonDataPath;
         CSV_DATA_PATH = csvDataPath;
@@ -157,6 +176,10 @@
         loadParquetFiles();
         LOGGER.info("Parquet files added successfully");
 
+        LOGGER.info("Adding Avro files to the bucket");
+        loadAvroFiles();
+        LOGGER.info("Avro files added successfully");
+
         LOGGER.info("Files added successfully");
     }
 
@@ -323,11 +346,37 @@
                 IoUtil.getMatchingFiles(Paths.get(generatedDataBasePath + "/external-filter"), PARQUET_FILTER);
         for (File file : files) {
             String fileName = file.getName();
+            String fileParent = file.getParent();
             String externalFilterDefinition = file.getParent().substring(generatedDataBasePath.length() + 1) + "/";
             loadData(file.getParent(), "", fileName, "parquet-data/" + externalFilterDefinition, "", false, false);
         }
     }
 
+    private static void loadAvroFiles() {
+        String generatedDataBasePath = AVRO_GEN_BASEDIR;
+        String definition = AVRO_DEFINITION;
+        String definitionSegment = "";
+
+        loadData(generatedDataBasePath, "", "dummy_tweet.avro", definition, definitionSegment, false, false);
+        loadData(generatedDataBasePath, "", "id_age.avro", definition, definitionSegment, false, false);
+        loadData(generatedDataBasePath, "", "id_age-string.avro", definition, definitionSegment, false, false);
+        loadData(generatedDataBasePath, "", "id_name.avro", definition, definitionSegment, false, false);
+        loadData(generatedDataBasePath, "", "id_name_comment.avro", definition, definitionSegment, false, false);
+        loadData(generatedDataBasePath, "", "heterogeneous_1.avro", definition, definitionSegment, false, false);
+        loadData(generatedDataBasePath, "", "heterogeneous_2.avro", definition, definitionSegment, false, false);
+        loadData(generatedDataBasePath, "", "avro_type.avro", definition, definitionSegment, false, false);
+
+        Collection<File> files =
+                IoUtil.getMatchingFiles(Paths.get(generatedDataBasePath + "/external-filter"), AVRO_FILTER);
+        for (File file : files) {
+            String fileName = file.getName();
+            String fileParent = file.getParent();
+            String externalFilterDefinition = file.getParent().substring(generatedDataBasePath.length() + 1) + "/";
+            loadData(file.getParent(), "", fileName, "avro-data/" + externalFilterDefinition, "", false, false);
+        }
+        return;
+    }
+
     private static void loadDirectory(String dataBasePath, String rootPath, FilenameFilter filter) {
         File dir = new File(dataBasePath, rootPath);
         if (!dir.exists() || !dir.isDirectory()) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/avro/AvroFileConverterUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/avro/AvroFileConverterUtil.java
new file mode 100644
index 0000000..db12e7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/avro/AvroFileConverterUtil.java
@@ -0,0 +1,111 @@
+/*
+ * 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.test.external_dataset.avro;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileReader;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.Collection;
+
+import org.apache.asterix.test.external_dataset.parquet.JsonUtil;
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.Path;
+import org.apache.hyracks.api.util.IoUtil;
+
+import tech.allegro.schema.json2avro.converter.JsonAvroConverter;
+
+public class AvroFileConverterUtil {
+
+    public static final String DEFAULT_PARQUET_SRC_PATH = "data/hdfs/parquet";
+    public static final String AVRO_GEN_BASEDIR = "target" + File.separatorChar + "generated_avro_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 AvroFileConverterUtil() {
+    }
+
+    private static void convertToJsonAndWriteAvro(File jsonFile, Schema schema, Path avroFilePath) throws IOException {
+        File outputFile = new File(avroFilePath.toString());
+        File parentDir = outputFile.getParentFile();
+        if (!parentDir.exists() && !parentDir.mkdirs()) {
+            throw new IOException("Failed to create directory " + parentDir);
+        }
+        GenericDatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
+        try (DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
+            dataFileWriter.create(schema, new File(avroFilePath.toString()));
+            JsonAvroConverter converter = new JsonAvroConverter();
+            try (BufferedReader reader = new BufferedReader(new FileReader(jsonFile))) {
+                String line;
+                while ((line = reader.readLine()) != null) {
+                    GenericRecord record = converter.convertToGenericDataRecord(line.getBytes(), schema);
+                    dataFileWriter.append(record);
+                }
+            } catch (DataFileWriter.AppendWriteException e) {
+                System.err.println("Failed to append record to Avro file: " + e.getMessage());
+            }
+        }
+    }
+
+    public static void writeAvroFile(File jsonFile, Path avroPath) throws IOException {
+        FileInputStream schemaInputStream = new FileInputStream(jsonFile);
+        Schema schema = JsonUtil.inferSchema(schemaInputStream, "avro_schema", NUM_OF_RECORDS_SCHEMA);
+        convertToJsonAndWriteAvro(jsonFile, schema, avroPath);
+    }
+
+    public static void convertToAvroRecursively(File localDataRoot, String src, String dest, FilenameFilter filter,
+            int startIndex) throws IOException {
+        File destPath = new File(localDataRoot, dest);
+
+        File dir = new File(src);
+        if (!dir.exists() || !dir.isDirectory()) {
+            return;
+        }
+
+        Collection<File> files = IoUtil.getMatchingFiles(dir.toPath(), filter);
+        for (File file : files) {
+            String fileName = file.getName().substring(0, file.getName().indexOf(".")) + ".avro";
+            Path outputPath = new Path(
+                    Paths.get(destPath.getAbsolutePath(), file.getParent().substring(startIndex), fileName).toString());
+
+            writeAvroFile(file, outputPath);
+        }
+    }
+
+    public static void convertToAvro(File localDataRoot, String src, String dest) throws IOException {
+        File srcPath = new File(localDataRoot, src);
+        File destPath = new File(localDataRoot, dest);
+
+        //write avro files
+        File[] listOfFiles = srcPath.listFiles();
+        for (File jsonFile : listOfFiles) {
+            String fileName = jsonFile.getName().substring(0, jsonFile.getName().indexOf(".")) + ".avro";
+            Path outputPath = new Path(destPath.getAbsolutePath(), fileName);
+            writeAvroFile(jsonFile, outputPath);
+        }
+        AvroFileExampleGeneratorUtil.writeExample();
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/avro/AvroFileExampleGeneratorUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/avro/AvroFileExampleGeneratorUtil.java
new file mode 100644
index 0000000..d62d2d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/avro/AvroFileExampleGeneratorUtil.java
@@ -0,0 +1,118 @@
+/*
+ * 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.test.external_dataset.avro;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.specific.SpecificDatumWriter;
+
+public class AvroFileExampleGeneratorUtil {
+    private static final String SCHEMA_STRING = "{\n" + "  \"type\": \"record\",\n" + "  \"name\": \"SimpleRecord\",\n"
+            + "  \"namespace\": \"com.example\",\n" + "  \"fields\": [\n" + "    {\n"
+            + "      \"name\": \"unionField\",\n" + "      \"type\": [\"int\", \"string\"],\n"
+            + "      \"doc\": \"This field can be either an int or a string.\"\n" + "    },\n" + "    {\n"
+            + "      \"name\": \"mapField\",\n" + "      \"type\": {\n" + "        \"type\": \"map\",\n"
+            + "        \"values\": \"int\",\n" + "        \"doc\": \"This is a map of string keys to int values.\"\n"
+            + "      },\n" + "      \"doc\": \"This field represents a map with string keys and integer values.\"\n"
+            + "    },\n" + "    {\n" + "      \"name\": \"nestedRecord\",\n" + "      \"type\": {\n"
+            + "        \"type\": \"record\",\n" + "        \"name\": \"NestedRecord\",\n" + "        \"fields\": [\n"
+            + "          {\n" + "            \"name\": \"nestedInt\",\n" + "            \"type\": \"int\"\n"
+            + "          },\n" + "          {\n" + "            \"name\": \"nestedString\",\n"
+            + "            \"type\": \"string\"\n" + "          }\n" + "        ]\n" + "      },\n"
+            + "      \"doc\": \"This is a nested record.\"\n" + "    },\n" + "    {\n"
+            + "      \"name\": \"booleanField\",\n" + "      \"type\": \"boolean\",\n"
+            + "      \"doc\": \"This is a boolean field.\"\n" + "    },\n" + "    {\n"
+            + "      \"name\": \"intField\",\n" + "      \"type\": \"int\",\n"
+            + "      \"doc\": \"This is an int field.\"\n" + "    },\n" + "    {\n" + "      \"name\": \"longField\",\n"
+            + "      \"type\": \"long\",\n" + "      \"doc\": \"This is a long field.\"\n" + "    },\n" + "    {\n"
+            + "      \"name\": \"floatField\",\n" + "      \"type\": \"float\",\n"
+            + "      \"doc\": \"This is a float field.\"\n" + "    },\n" + "    {\n"
+            + "      \"name\": \"doubleField\",\n" + "      \"type\": \"double\",\n"
+            + "      \"doc\": \"This is a double field.\"\n" + "    },\n" + "    {\n"
+            + "      \"name\": \"bytesField\",\n" + "      \"type\": \"bytes\",\n"
+            + "      \"doc\": \"This is a bytes field.\"\n" + "    },\n" + "    {\n"
+            + "      \"name\": \"stringField\",\n" + "      \"type\": \"string\",\n"
+            + "      \"doc\": \"This is a string field.\"\n" + "    }\n" + "  ]\n" + "}\n";
+
+    private static final String AVRO_GEN_BASEDIR = "target/generated_avro_files";
+    private static final String FILE_NAME = "avro_type.avro";
+
+    public static void writeExample() throws IOException {
+        Schema schema = new Schema.Parser().parse(SCHEMA_STRING);
+        File destPath = new File(AVRO_GEN_BASEDIR);
+        File outputFile = new File(destPath, FILE_NAME);
+
+        DatumWriter<GenericRecord> datumWriter = new SpecificDatumWriter<>(schema);
+        try (DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
+            dataFileWriter.create(schema, outputFile);
+
+            // First record with unionField as int
+            GenericRecord nestedRecord = new GenericData.Record(schema.getField("nestedRecord").schema());
+            nestedRecord.put("nestedInt", 100);
+            nestedRecord.put("nestedString", "Inside Nested");
+
+            // First record with various fields
+            GenericRecord record = new GenericData.Record(schema);
+            record.put("unionField", 42);
+            Map<String, Integer> map = new HashMap<>();
+            map.put("key1", 1);
+            map.put("key2", 2);
+            record.put("mapField", map);
+            record.put("nestedRecord", nestedRecord);
+            record.put("booleanField", true);
+            record.put("intField", 32);
+            record.put("longField", 64L);
+            record.put("floatField", 1.0f);
+            record.put("doubleField", 2.0);
+            record.put("bytesField", ByteBuffer.wrap(new byte[] { 0x01, 0x02 }));
+            record.put("stringField", "Example string");
+            dataFileWriter.append(record);
+
+            //second record to be added
+            GenericRecord record2 = new GenericData.Record(schema);
+            record2.put("unionField", "Example string");
+            Map<String, Integer> map2 = new HashMap<>();
+            map2.put("key3", 3);
+            map2.put("key4", 4);
+            record2.put("mapField", map2);
+            record2.put("nestedRecord", nestedRecord);
+            record2.put("booleanField", false);
+            record2.put("intField", 54);
+            record2.put("longField", 60L);
+            record2.put("floatField", 3.6f);
+            record2.put("doubleField", 5.77777);
+            record2.put("bytesField", ByteBuffer.wrap(new byte[] { 0x06, 0x04 }));
+            record2.put("stringField", "Sample Values");
+            dataFileWriter.append(record2);
+        } catch (IOException e) {
+            System.err.println("Failed to write AVRO file: " + e.getMessage());
+            e.printStackTrace();
+        }
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java
index 7892a17..7912d57 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.test.external_dataset.aws;
 
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createAvroFiles;
+import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createAvroFilesRecursively;
 import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createBinaryFiles;
 import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.createBinaryFilesRecursively;
 import static org.apache.asterix.test.external_dataset.ExternalDatasetTestUtils.setDataPaths;
@@ -194,6 +196,8 @@
         LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
         createBinaryFiles(DEFAULT_PARQUET_SRC_PATH);
         createBinaryFilesRecursively(EXTERNAL_FILTER_DATA_PATH);
+        createAvroFiles(DEFAULT_PARQUET_SRC_PATH);
+        createAvroFilesRecursively(EXTERNAL_FILTER_DATA_PATH);
         setNcEndpoints(testExecutor);
         startAwsS3MockServer();
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/microsoft/AzureBlobStorageExternalDatasetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/microsoft/AzureBlobStorageExternalDatasetTest.java
index 9858e56..d35440e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/microsoft/AzureBlobStorageExternalDatasetTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/microsoft/AzureBlobStorageExternalDatasetTest.java
@@ -127,6 +127,8 @@
         final TestExecutor testExecutor = new AzureTestExecutor();
         ExternalDatasetTestUtils.createBinaryFiles(PARQUET_RAW_DATA_PATH);
         createBinaryFilesRecursively(EXTERNAL_FILTER_DATA_PATH);
+        ExternalDatasetTestUtils.createAvroFiles(PARQUET_RAW_DATA_PATH);
+        createAvroFilesRecursively(EXTERNAL_FILTER_DATA_PATH);
         LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
         setNcEndpoints(testExecutor);
         createBlobServiceClient();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.01.ddl.sqlpp
similarity index 66%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.01.ddl.sqlpp
index a4fa97b..606c781 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.01.ddl.sqlpp
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Array access pushdown
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*dummy_tweet.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.02.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.02.query.sqlpp
index 7a863f7..f07bc35 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.02.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Push down get-item
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+SELECT a.entities.urls[0].display_url
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.03.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.03.query.sqlpp
index 7a863f7..e1abe7d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.03.query.sqlpp
@@ -16,10 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Push down a.entities.urls
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+SELECT a.entities.urls[*].display_url
+FROM AvroDataset a
+WHERE a.entities.urls IS NOT MISSING
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.04.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.04.query.sqlpp
index 7a863f7..4ebcca8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.04.query.sqlpp
@@ -16,10 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Push down scan-collection from unnest
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+
+SELECT urls.display_url
+FROM AvroDataset a, a.entities.urls urls
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.05.query.sqlpp
similarity index 75%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.05.query.sqlpp
index 7a863f7..951039c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.05.query.sqlpp
@@ -16,10 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Push down scan-collection from unnest in a subplan
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+
+SELECT VALUE COUNT(*)
+FROM AvroDataset a
+WHERE (EVERY ht in a.entities.urls SATISFIES ht.display_url = "string");
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.06.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.06.query.sqlpp
index 7a863f7..87827c3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/array-access/array-access.06.query.sqlpp
@@ -16,10 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Push down nested get_item
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+
+SELECT VALUE a.place.bounding_box.coordinates[0][0][0]
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.01.ddl.sqlpp
similarity index 66%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.01.ddl.sqlpp
index a4fa97b..65a2b38 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.01.ddl.sqlpp
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Field access pushdown
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*avro_type.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.02.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.02.query.sqlpp
index 7a863f7..61ed57e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.02.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+SELECT VALUE a.mapField
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.03.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.03.query.sqlpp
index 7a863f7..b97d9f8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-map/avro-map.03.query.sqlpp
@@ -16,10 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+SELECT RAW a.mapField.key1
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.01.ddl.sqlpp
similarity index 66%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.01.ddl.sqlpp
index a4fa97b..65a2b38 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.01.ddl.sqlpp
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Field access pushdown
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*avro_type.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.02.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.02.query.sqlpp
index 7a863f7..deef7c1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.02.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+SELECT VALUE a.nestedRecord
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.03.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.03.query.sqlpp
index 7a863f7..7e58df9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.03.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+SELECT VALUE a.nestedRecord.nestedInt
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.01.ddl.sqlpp
similarity index 66%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.01.ddl.sqlpp
index a4fa97b..65a2b38 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.01.ddl.sqlpp
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Field access pushdown
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*avro_type.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.02.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.02.query.sqlpp
index 7a863f7..136107f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.02.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+SELECT VALUE a.booleanField
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.03.query.sqlpp
similarity index 76%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.03.query.sqlpp
index 7a863f7..d238d38 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.03.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+SELECT a.intField,a.longField,a.floatField,a.doubleField
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-union/avro-union.01.ddl.sqlpp
similarity index 66%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-union/avro-union.01.ddl.sqlpp
index a4fa97b..65a2b38 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-union/avro-union.01.ddl.sqlpp
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Field access pushdown
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*avro_type.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-union/avro-union.02.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-union/avro-union.02.query.sqlpp
index 7a863f7..48d4587 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/avro-types/avro-union/avro-union.02.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+SELECT VALUE a.unionField
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.01.ddl.sqlpp
new file mode 100644
index 0000000..5e30b26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.01.ddl.sqlpp
@@ -0,0 +1,68 @@
+/*
+ * 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  : Field access pushdown
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*id_age.avro"),
+  ("format" = "avro")
+);
+
+CREATE EXTERNAL DATASET AvroDataset2(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*id_age.avro"),
+  ("format" = "avro")
+);
+
+CREATE EXTERNAL DATASET AvroDataset3(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*id_name_comment.avro"),
+  ("format" = "avro")
+);
+
+CREATE EXTERNAL DATASET AvroDataset4(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*dummy_tweet.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.02.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.02.query.sqlpp
index 7a863f7..9225aba 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.02.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+SELECT VALUE a
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.03.query.sqlpp
similarity index 76%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.03.query.sqlpp
index 7a863f7..3f70bc0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.03.query.sqlpp
@@ -16,10 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Ignore Field Access pushdown when requesting all fields
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+SELECT a1, a2.id
+FROM AvroDataset a1, AvroDataset2 a2
+WHERE a1.id = a2.id
+ORDER BY a2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.04.query.sqlpp
similarity index 76%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.04.query.sqlpp
index 7a863f7..dca1a16 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.04.query.sqlpp
@@ -16,10 +16,17 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Access different fields when joining two datasets
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+
+SELECT a1.age, a2.name
+FROM AvroDataset a1, AvroDataset3 a2
+WHERE a1.id = a2.id
+ORDER BY a2.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.05.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.05.query.sqlpp
index 7a863f7..33232b1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.05.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Push down children of common fields access when requesting nested values
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+SELECT a.user.id, a.user.name
+FROM AvroDataset4 a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.06.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.06.query.sqlpp
index 7a863f7..7b96d53 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/field-access/field-access.06.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Test SELECT COUNT(*)
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+
+
+SELECT VALUE COUNT(*)
+FROM AvroDataset4 a;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.1.ddl.sqlpp
similarity index 66%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.1.ddl.sqlpp
index a4fa97b..ce5eb8a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.1.ddl.sqlpp
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Heterogeneous access DDL
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*heterogeneous*"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.2.query.sqlpp
similarity index 72%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.2.query.sqlpp
index a4fa97b..bd5b240 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.2.query.sqlpp
@@ -16,16 +16,20 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Accessing a heterogeneous value
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
+USE test;
 
-import java.io.Serializable;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+SELECT VALUE (
+    CASE WHEN is_array(a.arrayOrObject) THEN
+        a.arrayOrObject[*].text
+    ELSE
+        a.arrayOrObject.text
+    END
+)
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.000.ddl.sqlpp
similarity index 73%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.000.ddl.sqlpp
index 7a863f7..a3a6d0d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.000.ddl.sqlpp
@@ -16,10 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-    char getSeparator();
-}
+USE test;
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="json-data/reviews/single-line/json"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.001.query.sqlpp
similarity index 72%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.001.query.sqlpp
index 7a863f7..1334f7f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.001.query.sqlpp
@@ -1,6 +1,6 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
+ * or more contiributor 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
@@ -16,10 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+SELECT VALUE COUNT(*)
+FROM AvroDataset a
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.999.ddl.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.999.ddl.sqlpp
index 7a863f7..20dc6fd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-avro-files/test.999.ddl.sqlpp
@@ -16,10 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
-
-    char getSeparator();
-}
+DROP DATAVERSE test IF EXISTS;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-type/invalid-type.1.ddl.sqlpp
similarity index 60%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-type/invalid-type.1.ddl.sqlpp
index a4fa97b..c6ee970 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/invalid-type/invalid-type.1.ddl.sqlpp
@@ -16,16 +16,27 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Test type validation for Avro
+* Expected Res : ASX3123: Type 'AvroType' contains declared fields, which is not supported for 'avro' format
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+CREATE TYPE AvroType as {
+  id: string,
+  text: string
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*id_age.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/missing-fields/missing-fields.1.ddl.sqlpp
similarity index 64%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/missing-fields/missing-fields.1.ddl.sqlpp
index a4fa97b..3c8e934 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/missing-fields/missing-fields.1.ddl.sqlpp
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Requesting non-existing fields should not fail
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*dummy_tweet.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/missing-fields/missing-fields.2.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/missing-fields/missing-fields.2.query.sqlpp
index 7a863f7..09fbb2d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/missing-fields/missing-fields.2.query.sqlpp
@@ -16,10 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Requesting non-existing fields should not fail
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
+USE test;
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
-
-    char getSeparator();
-}
+SELECT a.not_a_field1 IS MISSING as f1, a.user.not_a_field2 IS MISSING as f2
+FROM AvroDataset a
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp
new file mode 100644
index 0000000..67e38d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.1.ddl.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * 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 Avro files with different schemas
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*id_age.avro"),
+  ("include#1"="*id_name.avro"),
+  ("format" = "avro")
+);
+
+CREATE EXTERNAL DATASET AvroDataset2(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*id_age.avro"),
+  ("include#1"="*id_age-string.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.2.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.2.query.sqlpp
index 7a863f7..1df5ada 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.2.query.sqlpp
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Retrieve all fields from different Avro files with different schemas
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
+USE test;
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
-
-    char getSeparator();
-}
+SELECT VALUE a
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.3.query.sqlpp
similarity index 75%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.3.query.sqlpp
index 7a863f7..a9b28f6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.3.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Retrieve all fields from different Avro files with different schemas
+                  with conflicting fields
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+SELECT VALUE a
+FROM AvroDataset2 a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.4.query.sqlpp
similarity index 75%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.4.query.sqlpp
index 7a863f7..e84e089 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.4.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+ * Description  : Retrieve all fields from different Avro files with different schemas
+                  with conflicting fields
+ * Expected Res : Success
+ * Date         : Feb 23rd 2024
+ */
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+USE test;
 
-    char getSeparator();
-}
+SELECT VALUE a.age
+FROM AvroDataset2 a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/no-files/no-files.1.ddl.sqlpp
similarity index 62%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/no-files/no-files.1.ddl.sqlpp
index a4fa97b..507d6fa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/no-files/no-files.1.ddl.sqlpp
@@ -16,16 +16,27 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : No files
+* Expected Res : Warning: The provided external dataset configuration returned no files from the external source
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+-- param max-warnings:json=1000
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="NOT_A_DEFINITION"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/no-files/no-files.2.query.sqlpp
similarity index 75%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/no-files/no-files.2.query.sqlpp
index 7a863f7..6677ed6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/no-files/no-files.2.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : No files
+* Expected Res : Warning: The provided external dataset configuration returned no files from the external source
+* Date         : Feb 23rd 2024
+*/
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+-- param max-warnings:json=1000
 
-    char getSeparator();
-}
+USE test;
+
+SELECT VALUE COUNT(*)
+FROM AvroDataset a
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/object-concat/object-concat.1.ddl.sqlpp
similarity index 65%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/object-concat/object-concat.1.ddl.sqlpp
index a4fa97b..c7f127b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/object-concat/object-concat.1.ddl.sqlpp
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Concat two objects after pushdown
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*dummy_tweet.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/object-concat/object-concat.2.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/object-concat/object-concat.2.query.sqlpp
index 7a863f7..801daa3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/object-concat/object-concat.2.query.sqlpp
@@ -16,10 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Concat two objects after pushdown
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
+USE test;
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
 
-    char getSeparator();
-}
+SELECT VALUE object_concat(a.coordinates, a.user).name
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-all-fields/select-all-fields.1.ddl.sqlpp
similarity index 65%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-all-fields/select-all-fields.1.ddl.sqlpp
index a4fa97b..6d77dab 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-all-fields/select-all-fields.1.ddl.sqlpp
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Retrieve all fields from an Avro file
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+%template%,
+("container"="playground"),
+("definition"="avro-data/reviews"),
+("include"="*dummy_tweet.avro"),
+("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-all-fields/select-all-fields.2.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-all-fields/select-all-fields.2.query.sqlpp
index 7a863f7..0d20181 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-all-fields/select-all-fields.2.query.sqlpp
@@ -16,10 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Retrieve all fields from an Avro file
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
+USE test;
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
-
-    char getSeparator();
-}
+SELECT VALUE a
+FROM AvroDataset a
+ORDER BY a.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-count-one-field/select-count-one-field.1.ddl.sqlpp
similarity index 65%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-count-one-field/select-count-one-field.1.ddl.sqlpp
index a4fa97b..052b127 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-count-one-field/select-count-one-field.1.ddl.sqlpp
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Retrieve the number of texts in all tweets
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*dummy_tweet.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-count-one-field/select-count-one-field.2.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-count-one-field/select-count-one-field.2.query.sqlpp
index 7a863f7..3728b23 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/select-count-one-field/select-count-one-field.2.query.sqlpp
@@ -16,10 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Retrieve the number of texts in all tweets
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
+USE test;
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
-
-    char getSeparator();
-}
+SELECT VALUE count(a.text)
+FROM AvroDataset a;
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp
similarity index 66%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp
index a4fa97b..17433a5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.1.ddl.sqlpp
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Test Standard UTF-8
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*id_name_comment.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.2.query.sqlpp
similarity index 78%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.2.query.sqlpp
index 7a863f7..7681ab3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.2.query.sqlpp
@@ -16,10 +16,14 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Test Standard UTF-8
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
+USE test;
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
 
-    char getSeparator();
-}
+SELECT VALUE array_count(split(trim(a.comment),"𩸽"))
+FROM AvroDataset a
+WHERE contains(a.comment, "𩸽");
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.1.ddl.sqlpp
similarity index 66%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.1.ddl.sqlpp
index a4fa97b..c8d7d21 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.1.ddl.sqlpp
@@ -16,16 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Type mismatch DDL
+* Expected Res : Success
+* Date         : Feb 23rd 2024
+*/
 
-import java.io.Serializable;
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+USE test;
+
+
+CREATE TYPE AvroType as {
+};
+
+CREATE EXTERNAL DATASET AvroDataset(AvroType) USING %adapter%
+(
+  %template%,
+  ("container"="playground"),
+  ("definition"="avro-data/reviews"),
+  ("include"="*dummy_tweet.avro"),
+  ("format" = "avro")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.2.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.2.query.sqlpp
index 7a863f7..a3bb6e9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.2.query.sqlpp
@@ -16,10 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Accessing an array as an object
+* Expected Res : Warning
+* Date         : Feb 23rd 2024
+*/
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+-- param max-warnings:json=1000
 
-    char getSeparator();
-}
+USE test;
+
+
+SELECT VALUE a.entities.urls.display_url IS MISSING
+FROM AvroDataset a
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.3.query.sqlpp
similarity index 75%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.3.query.sqlpp
index 7a863f7..79ee625 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/common/avro/type-mismatch/type-mismatch.3.query.sqlpp
@@ -16,10 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+/*
+* Description  : Accessing an array as an object
+* Expected Res : Warning
+* Date         : Feb 23rd 2024
+*/
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+-- param max-warnings:json=1000
 
-    char getSeparator();
-}
+USE test;
+
+SELECT VALUE a.place.bounding_box.coordinates[0][0].not_object IS MISSING
+FROM AvroDataset a;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.02.adm
new file mode 100644
index 0000000..18f3275
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.02.adm
@@ -0,0 +1,2 @@
+{ "display_url": "string" }
+{  }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.03.adm
new file mode 100644
index 0000000..695240b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.03.adm
@@ -0,0 +1 @@
+{ "display_url": [ "string" ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.04.adm
new file mode 100644
index 0000000..41c14f5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.04.adm
@@ -0,0 +1 @@
+{ "display_url": "string" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.05.adm
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.05.adm
@@ -0,0 +1 @@
+2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.06.adm
new file mode 100644
index 0000000..15eecd2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/array-access/array-access.06.adm
@@ -0,0 +1,2 @@
+1.1
+1.1
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-map/avro-map.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-map/avro-map.02.adm
new file mode 100644
index 0000000..5560bf9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-map/avro-map.02.adm
@@ -0,0 +1,2 @@
+{ "key1": 1, "key2": 2 }
+{ "key3": 3, "key4": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-map/avro-map.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-map/avro-map.03.adm
new file mode 100644
index 0000000..fe0b81f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-map/avro-map.03.adm
@@ -0,0 +1,2 @@
+1
+null
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.02.adm
new file mode 100644
index 0000000..db09d84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.02.adm
@@ -0,0 +1,2 @@
+{ "nestedInt": 100, "nestedString": "Inside Nested" }
+{ "nestedInt": 100, "nestedString": "Inside Nested" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.03.adm
new file mode 100644
index 0000000..4eb537c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-nested-records/avro-nested-records.03.adm
@@ -0,0 +1,2 @@
+100
+100
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.02.adm
new file mode 100644
index 0000000..d252328
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.02.adm
@@ -0,0 +1,2 @@
+true
+false
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.03.adm
new file mode 100644
index 0000000..8a642e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-primitives/avro-primitives.03.adm
@@ -0,0 +1,2 @@
+{ "intField": 32, "longField": 64, "floatField": 1.0, "doubleField": 2.0 }
+{ "intField": 54, "longField": 60, "floatField": 3.6, "doubleField": 5.77777 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-union/avro-union.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-union/avro-union.02.adm
new file mode 100644
index 0000000..15f8776
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/avro-types/avro-union/avro-union.02.adm
@@ -0,0 +1,2 @@
+42
+"Example string"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.02.adm
new file mode 100644
index 0000000..7e235c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.02.adm
@@ -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/external-dataset/common/avro/field-access/field-access.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.03.adm
new file mode 100644
index 0000000..aaefab9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.03.adm
@@ -0,0 +1,7 @@
+{ "a1": { "id": 8, "age": 10 }, "id": 8 }
+{ "a1": { "id": 9, "age": 20 }, "id": 9 }
+{ "a1": { "id": 10, "age": 30 }, "id": 10 }
+{ "a1": { "id": 11, "age": 40 }, "id": 11 }
+{ "a1": { "id": 12, "age": 50 }, "id": 12 }
+{ "a1": { "id": 13, "age": 60 }, "id": 13 }
+{ "a1": { "id": 14, "age": 70 }, "id": 14 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.04.adm
new file mode 100644
index 0000000..aa3f801
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.04.adm
@@ -0,0 +1 @@
+{ "age": 10, "name": "William" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.05.adm
new file mode 100644
index 0000000..a1ad24e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.05.adm
@@ -0,0 +1,2 @@
+{ "id": 1, "name": "string" }
+{ "id": 1, "name": "string" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.06.adm
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/field-access/field-access.06.adm
@@ -0,0 +1 @@
+2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.02.adm
new file mode 100644
index 0000000..1ebe993
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/heterogeneous-access/heterogeneous-access.02.adm
@@ -0,0 +1,6 @@
+[ "1", "2" ]
+[ "3", "4" ]
+[ "5", "6" ]
+"7"
+"8"
+"9"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/missing-fields/missing-fields.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/missing-fields/missing-fields.2.adm
new file mode 100644
index 0000000..8876910
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/missing-fields/missing-fields.2.adm
@@ -0,0 +1,2 @@
+{ "f1": true, "f2": true }
+{ "f1": true, "f2": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.2.adm
new file mode 100644
index 0000000..0e2b980
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.2.adm
@@ -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/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.3.adm
new file mode 100644
index 0000000..8df138d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.3.adm
@@ -0,0 +1,14 @@
+{ "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 }
+{ "id": 15, "age": "10" }
+{ "id": 16, "age": "20" }
+{ "id": 17, "age": "30" }
+{ "id": 18, "age": "40" }
+{ "id": 19, "age": "50" }
+{ "id": 20, "age": "60" }
+{ "id": 21, "age": "70" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.4.adm
new file mode 100644
index 0000000..9256ef5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/multi-file-multi-schema/multi-file-multi-schema.4.adm
@@ -0,0 +1,14 @@
+10
+20
+30
+40
+50
+60
+70
+"10"
+"20"
+"30"
+"40"
+"50"
+"60"
+"70"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/no-files/no-files.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/no-files/no-files.02.adm
new file mode 100644
index 0000000..c227083
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/no-files/no-files.02.adm
@@ -0,0 +1 @@
+0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/object-concat/object-concat.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/object-concat/object-concat.2.adm
new file mode 100644
index 0000000..1b425f7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/object-concat/object-concat.2.adm
@@ -0,0 +1,2 @@
+"string"
+"string"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/select-all-fields/select-all-fields.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/select-all-fields/select-all-fields.2.adm
new file mode 100644
index 0000000..53f2518
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/select-all-fields/select-all-fields.2.adm
@@ -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/external-dataset/common/avro/select-count-one-field/select-count-one-field.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/select-count-one-field/select-count-one-field.2.adm
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/select-count-one-field/select-count-one-field.2.adm
@@ -0,0 +1 @@
+2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.2.adm
new file mode 100644
index 0000000..6fb86be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/string-standard-utf8/string-standard-utf8.2.adm
@@ -0,0 +1,2 @@
+2
+301
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/type-mismatch/type-mismatch.02.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/type-mismatch/type-mismatch.02.adm
new file mode 100644
index 0000000..0be5d98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/type-mismatch/type-mismatch.02.adm
@@ -0,0 +1,2 @@
+true
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/type-mismatch/type-mismatch.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/type-mismatch/type-mismatch.03.adm
new file mode 100644
index 0000000..0be5d98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/common/avro/type-mismatch/type-mismatch.03.adm
@@ -0,0 +1,2 @@
+true
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
index db4b5a8..57474b3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_s3.xml
@@ -294,6 +294,111 @@
       </compilation-unit>
     </test-case>
     <!-- Parquet Tests End -->
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/avro-types/avro-map">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/avro-types/avro-map</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/avro-types/avro-nested-records">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/avro-types/avro-nested-records</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/avro-types/avro-primitives">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/avro-types/avro-primitives</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/avro-types/avro-union">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/avro-types/avro-union</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/type-mismatch">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/type-mismatch</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/string-standard-utf8">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/string-standard-utf8</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/select-all-fields">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/select-all-fields</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/select-count-one-field">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/select-count-one-field</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/array-access">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/array-access</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/field-access">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/field-access</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/heterogeneous-access">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/heterogeneous-access</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/invalid-avro-files">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">none</output-dir>
+        <source-location>false</source-location>
+        <expected-error>Not an Avro data file.</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/invalid-type">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>ASX3123: Type 'AvroType' contains declared fields, which is not supported for 'avro' format</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/missing-fields">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/missing-fields</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/multi-file-multi-schema">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/multi-file-multi-schema</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/no-files">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/no-files</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-dataset">
+      <compilation-unit name="common/avro/object-concat">
+        <placeholder name="adapter" value="S3" />
+        <output-dir compare="Text">common/avro/object-concat</output-dir>
+      </compilation-unit>
+    </test-case>
     <!-- Dynamic prefixes tests start -->
     <test-case FilePath="external-dataset/common/dynamic-prefixes">
       <compilation-unit name="embed-with-closed-type">
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/AbstractCloudExternalFileWriter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/AbstractCloudExternalFileWriter.java
index 73a0aeb..e2c09eb 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/AbstractCloudExternalFileWriter.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/AbstractCloudExternalFileWriter.java
@@ -28,8 +28,8 @@
 import org.apache.asterix.cloud.clients.ICloudClient;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.runtime.writer.IExternalFilePrinter;
 import org.apache.asterix.runtime.writer.IExternalFileWriter;
+import org.apache.asterix.runtime.writer.IExternalPrinter;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -39,7 +39,7 @@
 import com.google.common.base.Utf8;
 
 abstract class AbstractCloudExternalFileWriter implements IExternalFileWriter {
-    private final IExternalFilePrinter printer;
+    private final IExternalPrinter printer;
     private final ICloudClient cloudClient;
     private final String bucket;
     private final boolean partitionedPath;
@@ -48,7 +48,7 @@
     private final IWriteBufferProvider bufferProvider;
     private ICloudBufferedWriter bufferedWriter;
 
-    AbstractCloudExternalFileWriter(IExternalFilePrinter printer, ICloudClient cloudClient, String bucket,
+    AbstractCloudExternalFileWriter(IExternalPrinter printer, ICloudClient cloudClient, String bucket,
             boolean partitionedPath, IWarningCollector warningCollector, SourceLocation pathSourceLocation) {
         this.printer = printer;
         this.cloudClient = cloudClient;
@@ -118,7 +118,7 @@
             if (isSdkException(e)) {
                 throw RuntimeDataException.create(ErrorCode.EXTERNAL_SOURCE_ERROR, e, getMessageOrToString(e));
             }
-            throw e;
+            throw HyracksDataException.create(e);
         }
     }
 
@@ -132,7 +132,7 @@
             if (isSdkException(e)) {
                 throw RuntimeDataException.create(ErrorCode.EXTERNAL_SOURCE_ERROR, e, getMessageOrToString(e));
             }
-            throw e;
+            throw HyracksDataException.create(e);
         }
     }
 
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/S3ExternalFileWriter.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/S3ExternalFileWriter.java
index e896c05..30ae0fa 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/S3ExternalFileWriter.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/S3ExternalFileWriter.java
@@ -20,7 +20,7 @@
 
 import org.apache.asterix.cloud.clients.ICloudClient;
 import org.apache.asterix.external.util.ExternalDataConstants;
-import org.apache.asterix.runtime.writer.IExternalFilePrinter;
+import org.apache.asterix.runtime.writer.IExternalPrinter;
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
@@ -29,7 +29,7 @@
 final class S3ExternalFileWriter extends AbstractCloudExternalFileWriter {
     static int MAX_LENGTH_IN_BYTES = 1024;
 
-    S3ExternalFileWriter(IExternalFilePrinter printer, ICloudClient cloudClient, String bucket, boolean partitionedPath,
+    S3ExternalFileWriter(IExternalPrinter printer, ICloudClient cloudClient, String bucket, boolean partitionedPath,
             IWarningCollector warningCollector, SourceLocation pathSourceLocation) {
         super(printer, cloudClient, bucket, partitionedPath, warningCollector, pathSourceLocation);
     }
diff --git a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/S3ExternalFileWriterFactory.java b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/S3ExternalFileWriterFactory.java
index 5036fc8..4477b1f 100644
--- a/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/S3ExternalFileWriterFactory.java
+++ b/asterixdb/asterix-cloud/src/main/java/org/apache/asterix/cloud/writer/S3ExternalFileWriterFactory.java
@@ -36,12 +36,12 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.aws.s3.S3Utils;
-import org.apache.asterix.runtime.writer.ExternalFileWriterConfiguration;
-import org.apache.asterix.runtime.writer.IExternalFileFilterWriterFactoryProvider;
-import org.apache.asterix.runtime.writer.IExternalFilePrinter;
-import org.apache.asterix.runtime.writer.IExternalFilePrinterFactory;
+import org.apache.asterix.runtime.writer.ExternalWriterConfiguration;
 import org.apache.asterix.runtime.writer.IExternalFileWriter;
 import org.apache.asterix.runtime.writer.IExternalFileWriterFactory;
+import org.apache.asterix.runtime.writer.IExternalFileWriterFactoryProvider;
+import org.apache.asterix.runtime.writer.IExternalPrinter;
+import org.apache.asterix.runtime.writer.IExternalPrinterFactory;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -59,24 +59,23 @@
     private static final long serialVersionUID = 4551318140901866805L;
     private static final Logger LOGGER = LogManager.getLogger();
     static final char SEPARATOR = '/';
-    public static final IExternalFileFilterWriterFactoryProvider PROVIDER =
-            new IExternalFileFilterWriterFactoryProvider() {
-                @Override
-                public IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration) {
-                    return new S3ExternalFileWriterFactory(configuration);
-                }
+    public static final IExternalFileWriterFactoryProvider PROVIDER = new IExternalFileWriterFactoryProvider() {
+        @Override
+        public IExternalFileWriterFactory create(ExternalWriterConfiguration configuration) {
+            return new S3ExternalFileWriterFactory(configuration);
+        }
 
-                @Override
-                public char getSeparator() {
-                    return SEPARATOR;
-                }
-            };
+        @Override
+        public char getSeparator() {
+            return SEPARATOR;
+        }
+    };
     private final Map<String, String> configuration;
     private final SourceLocation pathSourceLocation;
     private final String staticPath;
     private transient S3CloudClient cloudClient;
 
-    private S3ExternalFileWriterFactory(ExternalFileWriterConfiguration externalConfig) {
+    private S3ExternalFileWriterFactory(ExternalWriterConfiguration externalConfig) {
         configuration = externalConfig.getConfiguration();
         pathSourceLocation = externalConfig.getPathSourceLocation();
         staticPath = externalConfig.getStaticPath();
@@ -84,11 +83,11 @@
     }
 
     @Override
-    public IExternalFileWriter createWriter(IHyracksTaskContext context, IExternalFilePrinterFactory printerFactory)
+    public IExternalFileWriter createWriter(IHyracksTaskContext context, IExternalPrinterFactory printerFactory)
             throws HyracksDataException {
         buildClient();
         String bucket = configuration.get(ExternalDataConstants.CONTAINER_NAME_FIELD_NAME);
-        IExternalFilePrinter printer = printerFactory.createPrinter();
+        IExternalPrinter printer = printerFactory.createPrinter();
         IWarningCollector warningCollector = context.getWarningCollector();
         return new S3ExternalFileWriter(printer, cloudClient, bucket, staticPath == null, warningCollector,
                 pathSourceLocation);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index e5c137e..ef54ba8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -412,6 +412,8 @@
     PARSER_DATA_PARSER_UNEXPECTED_TOKEN(3120),
     REQUIRED_PARAM_OR_PARAM_IF_PARAM_IS_PRESENT(3121),
     PARAM_NOT_ALLOWED_IF_PARAM_IS_PRESENT(3122),
+    // Avro error
+    UNSUPPORTED_TYPE_FOR_AVRO(3123),
 
     // Lifecycle management errors
     DUPLICATE_PARTITION_ID(4000),
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index b3eb3a3..45b6185 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -417,6 +417,7 @@
 3120 = Unexpected token %s: was expecting %s
 3121 = Parameter '%1$s' or '%2$s' is required if '%3$s' is provided
 3122 = Parameter '%1$s' is not allowed if '%2$s' is provided
+3123 = Type '%1$s' contains declared fields, which is not supported for 'avro' format
 
 # Lifecycle management errors
 4000 = Partition id %1$s for node %2$s already in use by node %3$s
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AvroDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AvroDataParser.java
index d1744eb..d760c1f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AvroDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AvroDataParser.java
@@ -18,10 +18,14 @@
  */
 package org.apache.asterix.external.parser;
 
+import static org.apache.avro.Schema.Type.NULL;
+
 import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collection;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.asterix.builders.IARecordBuilder;
 import org.apache.asterix.builders.IAsterixListBuilder;
@@ -35,6 +39,7 @@
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IMutableValueStorage;
@@ -57,13 +62,15 @@
     }
 
     private final void parseObject(GenericRecord record, DataOutput out) throws IOException {
-        Schema schema = record.getSchema();
         IMutableValueStorage valueBuffer = parserContext.enterObject();
         IARecordBuilder objectBuilder = parserContext.getObjectBuilder(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        Schema schema = record.getSchema();
         for (Schema.Field field : schema.getFields()) {
-            valueBuffer.reset();
-            parseValue(field.schema(), record.get(field.name()), valueBuffer.getDataOutput());
-            objectBuilder.addField(parserContext.getSerializedFieldName(field.name()), valueBuffer);
+            if (record.get(field.name()) != null) {
+                valueBuffer.reset();
+                parseValue(field.schema(), record.get(field.name()), valueBuffer.getDataOutput());
+                objectBuilder.addField(parserContext.getSerializedFieldName(field.name()), valueBuffer);
+            }
         }
         objectBuilder.write(out, true);
         parserContext.exitObject(valueBuffer, null, objectBuilder);
@@ -83,6 +90,58 @@
         parserContext.exitCollection(valueBuffer, arrayBuilder);
     }
 
+    private void parseMap(Schema mapSchema, Map<String, ?> map, DataOutput out) throws IOException {
+        Schema valueSchema = mapSchema.getValueType();
+        final IMutableValueStorage valueBuffer = parserContext.enterCollection();
+        final IMutableValueStorage keyBuffer = parserContext.enterCollection();
+        IARecordBuilder objectBuilder = parserContext.getObjectBuilder(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        for (Map.Entry<String, ?> entry : map.entrySet()) {
+            keyBuffer.reset();
+            valueBuffer.reset();
+            serializeString(entry.getKey(), Schema.Type.STRING, keyBuffer.getDataOutput());
+            parseValue(valueSchema, entry.getValue(), valueBuffer.getDataOutput());
+            objectBuilder.addField(keyBuffer, valueBuffer);
+        }
+        objectBuilder.write(out, true);
+        parserContext.exitObject(valueBuffer, null, objectBuilder);
+    }
+
+    private final void parseUnion(Schema unionSchema, Object value, DataOutput out) throws IOException {
+        List<Schema> possibleTypes = unionSchema.getTypes();
+        for (Schema possibleType : possibleTypes) {
+            Schema.Type schemaType = possibleType.getType();
+            if (possibleType.getType() != NULL) {
+                if (matchesType(value, schemaType)) {
+                    parseValue(possibleType, value, out);
+                    return;
+                }
+            }
+        }
+    }
+
+    private boolean matchesType(Object value, Schema.Type schemaType) {
+        switch (schemaType) {
+            case INT:
+                return value instanceof Integer;
+            case STRING:
+                return value instanceof CharSequence;
+            case LONG:
+                return value instanceof Long;
+            case FLOAT:
+                return value instanceof Float;
+            case DOUBLE:
+                return value instanceof Double;
+            case BOOLEAN:
+                return value instanceof Boolean;
+            case BYTES:
+                return value instanceof Byte;
+            case RECORD:
+                return value instanceof GenericData.Record;
+            default:
+                return false;
+        }
+    }
+
     private void parseValue(Schema schema, Object value, DataOutput out) throws IOException {
         Schema.Type type = schema.getType();
         switch (type) {
@@ -92,8 +151,12 @@
             case ARRAY:
                 parseArray(schema, (Collection<?>) value, out);
                 break;
-            case MAP:
             case UNION:
+                parseUnion(schema, value, out);
+                break;
+            case MAP:
+                parseMap(schema, (Map<String, ?>) value, out);
+                break;
             case ENUM:
             case FIXED:
             case NULL:
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 189e0d8..ae3b567 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
@@ -917,6 +917,19 @@
                 || ExternalDataConstants.FORMAT_PARQUET.equals(properties.get(ExternalDataConstants.KEY_FORMAT));
     }
 
+    public static void validateAvroTypeAndConfiguration(Map<String, String> properties, ARecordType datasetRecordType)
+            throws CompilationException {
+        if (isAvroFormat(properties)) {
+            if (datasetRecordType.getFieldTypes().length != 0) {
+                throw new CompilationException(ErrorCode.UNSUPPORTED_TYPE_FOR_AVRO, datasetRecordType.getTypeName());
+            }
+        }
+    }
+
+    public static boolean isAvroFormat(Map<String, String> properties) {
+        return ExternalDataConstants.FORMAT_AVRO.equals(properties.get(ExternalDataConstants.KEY_FORMAT));
+    }
+
     public static void setExternalDataProjectionInfo(ExternalDatasetProjectionFiltrationInfo projectionInfo,
             Map<String, String> properties) throws IOException {
         properties.put(ExternalDataConstants.KEY_REQUESTED_FIELDS,
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/LocalFSExternalFileWriter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/LocalFSExternalFileWriter.java
index a3a2f70..4166dde 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/LocalFSExternalFileWriter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/LocalFSExternalFileWriter.java
@@ -25,19 +25,19 @@
 
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.runtime.writer.IExternalFilePrinter;
 import org.apache.asterix.runtime.writer.IExternalFileWriter;
+import org.apache.asterix.runtime.writer.IExternalPrinter;
 import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IValueReference;
 
 final class LocalFSExternalFileWriter implements IExternalFileWriter {
-    private final IExternalFilePrinter printer;
+    private final IExternalPrinter printer;
     private final ILocalFSValidator validator;
     private final SourceLocation pathSourceLocation;
 
-    LocalFSExternalFileWriter(IExternalFilePrinter printer, ILocalFSValidator validator,
+    LocalFSExternalFileWriter(IExternalPrinter printer, ILocalFSValidator validator,
             SourceLocation pathSourceLocation) {
         this.printer = printer;
         this.validator = validator;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/LocalFSExternalFileWriterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/LocalFSExternalFileWriterFactory.java
index 313757a..73f34f1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/LocalFSExternalFileWriterFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/LocalFSExternalFileWriterFactory.java
@@ -23,11 +23,11 @@
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.runtime.writer.ExternalFileWriterConfiguration;
-import org.apache.asterix.runtime.writer.IExternalFileFilterWriterFactoryProvider;
-import org.apache.asterix.runtime.writer.IExternalFilePrinterFactory;
+import org.apache.asterix.runtime.writer.ExternalWriterConfiguration;
 import org.apache.asterix.runtime.writer.IExternalFileWriter;
 import org.apache.asterix.runtime.writer.IExternalFileWriterFactory;
+import org.apache.asterix.runtime.writer.IExternalFileWriterFactoryProvider;
+import org.apache.asterix.runtime.writer.IExternalPrinterFactory;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -36,18 +36,17 @@
 public final class LocalFSExternalFileWriterFactory implements IExternalFileWriterFactory {
     private static final long serialVersionUID = 871685327574547749L;
     private static final char SEPARATOR = File.separatorChar;
-    public static final IExternalFileFilterWriterFactoryProvider PROVIDER =
-            new IExternalFileFilterWriterFactoryProvider() {
-                @Override
-                public IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration) {
-                    return new LocalFSExternalFileWriterFactory(configuration);
-                }
+    public static final IExternalFileWriterFactoryProvider PROVIDER = new IExternalFileWriterFactoryProvider() {
+        @Override
+        public IExternalFileWriterFactory create(ExternalWriterConfiguration configuration) {
+            return new LocalFSExternalFileWriterFactory(configuration);
+        }
 
-                @Override
-                public char getSeparator() {
-                    return SEPARATOR;
-                }
-            };
+        @Override
+        public char getSeparator() {
+            return SEPARATOR;
+        }
+    };
     private static final ILocalFSValidator NO_OP_VALIDATOR = LocalFSExternalFileWriterFactory::noOpValidation;
     private static final ILocalFSValidator VALIDATOR = LocalFSExternalFileWriterFactory::validate;
     private final SourceLocation pathSourceLocation;
@@ -55,7 +54,7 @@
     private final String staticPath;
     private boolean validated;
 
-    private LocalFSExternalFileWriterFactory(ExternalFileWriterConfiguration externalConfig) {
+    private LocalFSExternalFileWriterFactory(ExternalWriterConfiguration externalConfig) {
         pathSourceLocation = externalConfig.getPathSourceLocation();
         singleNodeCluster = externalConfig.isSingleNodeCluster();
         staticPath = externalConfig.getStaticPath();
@@ -63,7 +62,7 @@
     }
 
     @Override
-    public IExternalFileWriter createWriter(IHyracksTaskContext context, IExternalFilePrinterFactory printerFactory)
+    public IExternalFileWriter createWriter(IHyracksTaskContext context, IExternalPrinterFactory printerFactory)
             throws HyracksDataException {
         ILocalFSValidator validator = VALIDATOR;
         if (staticPath != null) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalFilePrinter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalFilePrinter.java
index 8f8c63a..57e7b58 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalFilePrinter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalFilePrinter.java
@@ -22,12 +22,12 @@
 import java.io.PrintStream;
 
 import org.apache.asterix.external.writer.compressor.IExternalFileCompressStreamFactory;
-import org.apache.asterix.runtime.writer.IExternalFilePrinter;
+import org.apache.asterix.runtime.writer.IExternalPrinter;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
 
-final class TextualExternalFilePrinter implements IExternalFilePrinter {
+final class TextualExternalFilePrinter implements IExternalPrinter {
     private final IPrinter printer;
     private final IExternalFileCompressStreamFactory compressStreamFactory;
     private TextualOutputStreamDelegate delegate;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalFilePrinterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalFilePrinterFactory.java
index 6778532..e3d0a66 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalFilePrinterFactory.java
@@ -19,23 +19,21 @@
 package org.apache.asterix.external.writer.printer;
 
 import org.apache.asterix.external.writer.compressor.IExternalFileCompressStreamFactory;
-import org.apache.asterix.runtime.writer.IExternalFilePrinter;
-import org.apache.asterix.runtime.writer.IExternalFilePrinterFactory;
+import org.apache.asterix.runtime.writer.IExternalPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
-public class TextualExternalFilePrinterFactory implements IExternalFilePrinterFactory {
-    private static final long serialVersionUID = 9155959967258587588L;
-    private final IPrinterFactory printerFactory;
+public class TextualExternalFilePrinterFactory extends TextualExternalPrinterFactory {
+    private static final long serialVersionUID = 8971234908711234L;
     private final IExternalFileCompressStreamFactory compressStreamFactory;
 
     public TextualExternalFilePrinterFactory(IPrinterFactory printerFactory,
             IExternalFileCompressStreamFactory compressStreamFactory) {
-        this.printerFactory = printerFactory;
+        super(printerFactory);
         this.compressStreamFactory = compressStreamFactory;
     }
 
     @Override
-    public IExternalFilePrinter createPrinter() {
+    public IExternalPrinter createPrinter() {
         return new TextualExternalFilePrinter(printerFactory.createPrinter(), compressStreamFactory);
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalPrinter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalPrinter.java
new file mode 100644
index 0000000..537af2e
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalPrinter.java
@@ -0,0 +1,64 @@
+/*
+ * 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.writer.printer;
+
+import java.io.OutputStream;
+import java.io.PrintStream;
+
+import org.apache.asterix.runtime.writer.IExternalPrinter;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+
+final class TextualExternalPrinter implements IExternalPrinter {
+    private final IPrinter printer;
+    private TextualOutputStreamDelegate delegate;
+    private PrintStream printStream;
+
+    TextualExternalPrinter(IPrinter printer) {
+        this.printer = printer;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        printer.init();
+    }
+
+    @Override
+    public void newStream(OutputStream outputStream) {
+        delegate = new TextualOutputStreamDelegate(outputStream);
+        printStream = new PrintStream(delegate);
+    }
+
+    @Override
+    public void print(IValueReference value) throws HyracksDataException {
+        printer.print(value.getByteArray(), value.getStartOffset(), value.getLength(), printStream);
+        delegate.checkError();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (printStream != null) {
+            printStream.close();
+            printStream = null;
+            delegate.checkError();
+            delegate = null;
+        }
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalPrinterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalPrinterFactory.java
new file mode 100644
index 0000000..d779793c
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/writer/printer/TextualExternalPrinterFactory.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.asterix.external.writer.printer;
+
+import org.apache.asterix.runtime.writer.IExternalPrinter;
+import org.apache.asterix.runtime.writer.IExternalPrinterFactory;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+public class TextualExternalPrinterFactory implements IExternalPrinterFactory {
+    private static final long serialVersionUID = 9155959967258587588L;
+    protected final IPrinterFactory printerFactory;
+
+    public TextualExternalPrinterFactory(IPrinterFactory printerFactory) {
+        this.printerFactory = printerFactory;
+    }
+
+    @Override
+    public IExternalPrinter createPrinter() {
+        return new TextualExternalPrinter(printerFactory.createPrinter());
+    }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CopyToStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CopyToStatement.java
index 2520755..599d528 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CopyToStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CopyToStatement.java
@@ -43,7 +43,7 @@
     private final Map<Integer, VariableExpr> partitionsVariables;
     private final List<OrderbyClause.OrderModifier> orderByModifiers;
     private final List<OrderbyClause.NullOrderModifier> orderByNullModifierList;
-    private final List<Expression> keyExpressions;
+    private List<Expression> keyExpressions;
     private final boolean autogenerated;
     private Namespace namespace;
     private Query query;
@@ -89,7 +89,7 @@
         this.orderByModifiers = orderByModifiers;
         this.orderByNullModifierList = orderByNullModifierList;
         this.varCounter = varCounter;
-        this.keyExpressions = keyExpressions;
+        this.keyExpressions = keyExpressions != null ? keyExpressions : new ArrayList<>();
         this.autogenerated = autogenerated;
 
         if (pathExpressions.isEmpty()) {
@@ -214,6 +214,7 @@
         topLevelExpressions.addAll(pathExpressions);
         topLevelExpressions.addAll(partitionExpressions);
         topLevelExpressions.addAll(orderByList);
+        topLevelExpressions.addAll(keyExpressions);
         return topLevelExpressions;
     }
 
@@ -231,15 +232,15 @@
         return keyExpressions;
     }
 
+    public void setKeyExpressions(List<Expression> keyExpressions) {
+        this.keyExpressions = keyExpressions;
+    }
+
     public boolean isAutogenerated() {
         return autogenerated;
     }
 
-    public boolean isSinkFileStore() {
+    public boolean isFileStoreSink() {
         return keyExpressions.isEmpty() && !autogenerated;
     }
-
-    public boolean isSinkDatabaseWithKey() {
-        return !keyExpressions.isEmpty() || autogenerated;
-    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
index 8c7b915..3975425 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
@@ -302,6 +302,10 @@
         changed |= order.first;
         stmtCopy.setOrderByList(order.second);
 
+        Pair<Boolean, List<Expression>> key = inlineUdfsInExprList(stmtCopy.getKeyExpressions());
+        changed |= key.first;
+        stmtCopy.setKeyExpressions(key.second);
+
         return changed;
     }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 52e2678..6151b02 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -573,12 +573,10 @@
         cto.getSourceVariable().accept(this, step);
         out.println();
 
-        if (cto.isSinkFileStore()) {
+        if (cto.isFileStoreSink()) {
             formatPrintCopyToFileStore(cto, step);
-        } else if (cto.isSinkDatabaseWithKey()) {
-            formatPrintCopyToDatabaseWithKey(cto, step);
         } else {
-            throw new IllegalStateException("NYI: This should never happen");
+            formatPrintCopyToDatabaseWithKey(cto, step);
         }
 
         out.println("with ");
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
index e3e2484..a7e1f90 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
@@ -437,6 +437,9 @@
         // Visit path exprs
         stmtCopy.setPathExpressions(visit(stmtCopy.getPathExpressions(), stmtCopy));
 
+        // Visit key exprs
+        stmtCopy.setKeyExpressions(visit(stmtCopy.getKeyExpressions(), stmtCopy));
+
         return null;
     }
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
index f3d2675..847393c 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
@@ -364,6 +364,7 @@
         stmtCopy.setPathExpressions(visit(stmtCopy.getPathExpressions(), arg));
         stmtCopy.setPartitionExpressions(visit(stmtCopy.getPartitionExpressions(), arg));
         stmtCopy.setOrderByList(visit(stmtCopy.getOrderByList(), arg));
+        stmtCopy.setKeyExpressions(visit(stmtCopy.getKeyExpressions(), arg));
         return null;
     }
 
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 8d8ca80..340eb0a 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
@@ -106,9 +106,9 @@
 import org.apache.asterix.runtime.operators.LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor;
 import org.apache.asterix.runtime.operators.LSMSecondaryUpsertOperatorDescriptor;
 import org.apache.asterix.runtime.operators.LSMSecondaryUpsertWithNestedPlanOperatorDescriptor;
-import org.apache.asterix.runtime.writer.ExternalWriterFactory;
-import org.apache.asterix.runtime.writer.IExternalFilePrinterFactory;
+import org.apache.asterix.runtime.writer.ExternalFileWriterFactory;
 import org.apache.asterix.runtime.writer.IExternalFileWriterFactory;
+import org.apache.asterix.runtime.writer.IExternalPrinterFactory;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -761,8 +761,8 @@
         fileWriterFactory.validate();
         String fileExtension = ExternalWriterProvider.getFileExtension(sink);
         int maxResult = ExternalWriterProvider.getMaxResult(sink);
-        IExternalFilePrinterFactory printerFactory = ExternalWriterProvider.createPrinter(sink, sourceType);
-        ExternalWriterFactory writerFactory = new ExternalWriterFactory(fileWriterFactory, printerFactory,
+        IExternalPrinterFactory printerFactory = ExternalWriterProvider.createPrinter(sink, sourceType);
+        ExternalFileWriterFactory writerFactory = new ExternalFileWriterFactory(fileWriterFactory, printerFactory,
                 fileExtension, maxResult, dynamicPathEvalFactory, staticPath, pathSourceLocation);
         SinkExternalWriterRuntimeFactory runtime = new SinkExternalWriterRuntimeFactory(sourceColumn, partitionColumns,
                 partitionComparatorFactories, inputDesc, writerFactory);
@@ -770,6 +770,13 @@
     }
 
     @Override
+    public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteDatabaseWithKeyRuntime(int sourceColumn,
+            IScalarEvaluatorFactory[] keyEvaluatorFactories, boolean autogenerated, IWriteDataSink sink,
+            RecordDescriptor inputDesc, Object sourceType) throws AlgebricksException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
             int[] printColumns, IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
             IResultSerializerFactoryProvider resultSerializerFactoryProvider, RecordDescriptor inputDesc,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/provider/ExternalWriterProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/provider/ExternalWriterProvider.java
index 9142556..2d77a94 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/provider/ExternalWriterProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/provider/ExternalWriterProvider.java
@@ -30,17 +30,17 @@
 import org.apache.asterix.external.writer.compressor.NoOpExternalFileCompressStreamFactory;
 import org.apache.asterix.external.writer.printer.TextualExternalFilePrinterFactory;
 import org.apache.asterix.formats.nontagged.CleanJSONPrinterFactoryProvider;
-import org.apache.asterix.runtime.writer.ExternalFileWriterConfiguration;
-import org.apache.asterix.runtime.writer.IExternalFileFilterWriterFactoryProvider;
-import org.apache.asterix.runtime.writer.IExternalFilePrinterFactory;
+import org.apache.asterix.runtime.writer.ExternalWriterConfiguration;
 import org.apache.asterix.runtime.writer.IExternalFileWriterFactory;
+import org.apache.asterix.runtime.writer.IExternalFileWriterFactoryProvider;
+import org.apache.asterix.runtime.writer.IExternalPrinterFactory;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IWriteDataSink;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 
 public class ExternalWriterProvider {
-    private static final Map<String, IExternalFileFilterWriterFactoryProvider> CREATOR_MAP;
+    private static final Map<String, IExternalFileWriterFactoryProvider> CREATOR_MAP;
     private static final Map<String, IExternalFileCompressStreamFactory> STREAM_COMPRESSORS;
 
     private ExternalWriterProvider() {
@@ -59,7 +59,7 @@
     public static IExternalFileWriterFactory createWriterFactory(ICcApplicationContext appCtx, IWriteDataSink sink,
             String staticPath, SourceLocation pathExpressionLocation) {
         String adapterName = sink.getAdapterName().toLowerCase();
-        IExternalFileFilterWriterFactoryProvider creator = CREATOR_MAP.get(adapterName);
+        IExternalFileWriterFactoryProvider creator = CREATOR_MAP.get(adapterName);
 
         if (creator == null) {
             throw new UnsupportedOperationException("Unsupported adapter " + adapterName);
@@ -83,12 +83,12 @@
         return Integer.parseInt(maxResultString);
     }
 
-    private static ExternalFileWriterConfiguration createConfiguration(ICcApplicationContext appCtx,
-            IWriteDataSink sink, String staticPath, SourceLocation pathExpressionLocation) {
+    private static ExternalWriterConfiguration createConfiguration(ICcApplicationContext appCtx, IWriteDataSink sink,
+            String staticPath, SourceLocation pathExpressionLocation) {
         Map<String, String> params = sink.getConfiguration();
         boolean singleNodeCluster = isSingleNodeCluster(appCtx);
 
-        return new ExternalFileWriterConfiguration(params, pathExpressionLocation, staticPath, singleNodeCluster);
+        return new ExternalWriterConfiguration(params, pathExpressionLocation, staticPath, singleNodeCluster);
     }
 
     private static boolean isSingleNodeCluster(ICcApplicationContext appCtx) {
@@ -96,8 +96,8 @@
         return ccs.getNodeManager().getIpAddressNodeNameMap().size() == 1;
     }
 
-    private static void addCreator(String adapterName, IExternalFileFilterWriterFactoryProvider creator) {
-        IExternalFileFilterWriterFactoryProvider registeredCreator = CREATOR_MAP.get(adapterName.toLowerCase());
+    private static void addCreator(String adapterName, IExternalFileWriterFactoryProvider creator) {
+        IExternalFileWriterFactoryProvider registeredCreator = CREATOR_MAP.get(adapterName.toLowerCase());
         if (registeredCreator != null) {
             throw new IllegalStateException(
                     "Adapter " + adapterName + " is registered to " + registeredCreator.getClass().getName());
@@ -105,7 +105,7 @@
         CREATOR_MAP.put(adapterName.toLowerCase(), creator);
     }
 
-    public static IExternalFilePrinterFactory createPrinter(IWriteDataSink sink, Object sourceType) {
+    public static IExternalPrinterFactory createPrinter(IWriteDataSink sink, Object sourceType) {
         Map<String, String> configuration = sink.getConfiguration();
         String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
 
@@ -131,7 +131,7 @@
     }
 
     public static char getSeparator(String adapterName) {
-        IExternalFileFilterWriterFactoryProvider creator = CREATOR_MAP.get(adapterName.toLowerCase());
+        IExternalFileWriterFactoryProvider creator = CREATOR_MAP.get(adapterName.toLowerCase());
 
         if (creator == null) {
             throw new UnsupportedOperationException("Unsupported adapter " + adapterName);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/DynamicPathResolver.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/DynamicPathResolver.java
index 7105efa..c6d1dbe 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/DynamicPathResolver.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/DynamicPathResolver.java
@@ -54,7 +54,7 @@
     @Override
     public String getPartitionDirectory(IFrameTupleReference tuple) throws HyracksDataException {
         if (!appendPrefix(tuple)) {
-            return ExternalWriter.UNRESOLVABLE_PATH;
+            return ExternalFileWriter.UNRESOLVABLE_PATH;
         }
 
         if (dirStringBuilder.length() > 0 && dirStringBuilder.charAt(dirStringBuilder.length() - 1) != fileSeparator) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalWriter.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriter.java
similarity index 94%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalWriter.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriter.java
index 5fc07af..f9f98da 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalWriter.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriter.java
@@ -23,7 +23,7 @@
 import org.apache.hyracks.data.std.api.IValueReference;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-final class ExternalWriter implements IExternalWriter {
+final class ExternalFileWriter implements IExternalWriter {
     static final String UNRESOLVABLE_PATH = "UNRESOLVABLE_PATH";
     private final IPathResolver pathResolver;
     private final IExternalFileWriter writer;
@@ -31,7 +31,7 @@
     private String partitionPath;
     private int tupleCounter;
 
-    public ExternalWriter(IPathResolver pathResolver, IExternalFileWriter writer, int maxResultPerFile) {
+    public ExternalFileWriter(IPathResolver pathResolver, IExternalFileWriter writer, int maxResultPerFile) {
         this.pathResolver = pathResolver;
         this.writer = writer;
         this.maxResultPerFile = maxResultPerFile;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalWriterFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriterFactory.java
similarity index 90%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalWriterFactory.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriterFactory.java
index e7c0db0..5981584 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalWriterFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriterFactory.java
@@ -28,17 +28,17 @@
 import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
-public class ExternalWriterFactory implements IExternalWriterFactory {
+public class ExternalFileWriterFactory implements IExternalWriterFactory {
     private static final long serialVersionUID = 1412969574113419638L;
     private final IExternalFileWriterFactory writerFactory;
-    private final IExternalFilePrinterFactory printerFactory;
+    private final IExternalPrinterFactory printerFactory;
     private final String fileExtension;
     private final int maxResult;
     private final IScalarEvaluatorFactory pathEvalFactory;
     private final String staticPath;
     private final SourceLocation pathSourceLocation;
 
-    public ExternalWriterFactory(IExternalFileWriterFactory writerFactory, IExternalFilePrinterFactory printerFactory,
+    public ExternalFileWriterFactory(IExternalFileWriterFactory writerFactory, IExternalPrinterFactory printerFactory,
             String fileExtension, int maxResult, IScalarEvaluatorFactory pathEvalFactory, String staticPath,
             SourceLocation pathSourceLocation) {
         this.writerFactory = writerFactory;
@@ -65,6 +65,6 @@
             resolver = new StaticPathResolver(fileExtension, fileSeparator, partition, staticPath);
         }
         IExternalFileWriter writer = writerFactory.createWriter(context, printerFactory);
-        return new ExternalWriter(resolver, writer, maxResult);
+        return new ExternalFileWriter(resolver, writer, maxResult);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriterConfiguration.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalWriterConfiguration.java
similarity index 91%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriterConfiguration.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalWriterConfiguration.java
index b62a07a..fbb05ee 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalFileWriterConfiguration.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/ExternalWriterConfiguration.java
@@ -22,13 +22,13 @@
 
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
-public final class ExternalFileWriterConfiguration {
+public final class ExternalWriterConfiguration {
     private final Map<String, String> configuration;
     private final SourceLocation pathSourceLocation;
     private final String staticPath;
     private final boolean singleNodeCluster;
 
-    public ExternalFileWriterConfiguration(Map<String, String> configuration, SourceLocation pathSourceLocation,
+    public ExternalWriterConfiguration(Map<String, String> configuration, SourceLocation pathSourceLocation,
             String staticPath, boolean singleNodeCluster) {
         this.configuration = configuration;
         this.pathSourceLocation = pathSourceLocation;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileWriterFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileWriterFactory.java
index d8f1f84..31dba79 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileWriterFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileWriterFactory.java
@@ -20,15 +20,14 @@
 
 import java.io.Serializable;
 
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * An interface for writing to a storage device
- * Implementer should also provide a singleton to {@link IExternalFileFilterWriterFactoryProvider}
+ * Implementer should also provide a singleton to {@link IExternalFileWriterFactoryProvider}
  */
-public interface IExternalFileWriterFactory extends Serializable {
+public interface IExternalFileWriterFactory extends IExternalWriterFactoryValidator, Serializable {
     /**
      * Create a writer
      *
@@ -36,16 +35,11 @@
      * @param printerFactory printer factory for writing the final result
      * @return a new file writer
      */
-    IExternalFileWriter createWriter(IHyracksTaskContext context, IExternalFilePrinterFactory printerFactory)
+    IExternalFileWriter createWriter(IHyracksTaskContext context, IExternalPrinterFactory printerFactory)
             throws HyracksDataException;
 
     /**
      * @return file (or path) separator
      */
     char getSeparator();
-
-    /**
-     * Validate the writer by running a test write routine to ensure the writer has the appropriate permissions
-     */
-    void validate() throws AlgebricksException;
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileWriterFactoryProvider.java
similarity index 85%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileWriterFactoryProvider.java
index 7a863f7..eabd2cb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileFilterWriterFactoryProvider.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFileWriterFactoryProvider.java
@@ -18,8 +18,8 @@
  */
 package org.apache.asterix.runtime.writer;
 
-public interface IExternalFileFilterWriterFactoryProvider {
-    IExternalFileWriterFactory create(ExternalFileWriterConfiguration configuration);
+public interface IExternalFileWriterFactoryProvider {
+    IExternalFileWriterFactory create(ExternalWriterConfiguration configuration);
 
     char getSeparator();
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinter.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalPrinter.java
similarity index 97%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinter.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalPrinter.java
index ba5fa1d..54fd152 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinter.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalPrinter.java
@@ -26,7 +26,7 @@
 /**
  * An {@link IExternalFileWriter} printer
  */
-public interface IExternalFilePrinter {
+public interface IExternalPrinter {
 
     /**
      * Open the printer
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalPrinterFactory.java
similarity index 86%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalPrinterFactory.java
index a4fa97b..4d9352a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalPrinterFactory.java
@@ -23,9 +23,9 @@
 /**
  * {@link IExternalFileWriter} printer factory
  */
-public interface IExternalFilePrinterFactory extends Serializable {
+public interface IExternalPrinterFactory extends Serializable {
     /**
-     * @return a new external file printer
+     * @return a new external printer
      */
-    IExternalFilePrinter createPrinter();
+    IExternalPrinter createPrinter();
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalWriterFactoryValidator.java
similarity index 76%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalWriterFactoryValidator.java
index a4fa97b..4a75db6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalWriterFactoryValidator.java
@@ -18,14 +18,11 @@
  */
 package org.apache.asterix.runtime.writer;
 
-import java.io.Serializable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
+public interface IExternalWriterFactoryValidator {
     /**
-     * @return a new external file printer
+     * Perform the necessary validation to ensure the writer has the proper permissions
      */
-    IExternalFilePrinter createPrinter();
+    void validate() throws AlgebricksException;
 }
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 2072dee..5240e0c 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
@@ -65,6 +65,10 @@
             SourceLocation pathSourceLocation, IWriteDataSink sink, RecordDescriptor inputDesc, Object sourceType)
             throws AlgebricksException;
 
+    Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteDatabaseWithKeyRuntime(int sourceColumn,
+            IScalarEvaluatorFactory[] keyEvaluatorFactories, boolean autogenerated, IWriteDataSink sink,
+            RecordDescriptor inputDesc, Object sourceType) throws AlgebricksException;
+
     Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink, int[] printColumns,
             IPrinterFactory[] printerFactories, IAWriterFactory writerFactory,
             IResultSerializerFactoryProvider resultSerializerFactoryProvider, RecordDescriptor inputDesc,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/WriteOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/WriteOperator.java
index 7eef90e..89a5148 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/WriteOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/WriteOperator.java
@@ -41,17 +41,21 @@
     private final Mutable<ILogicalExpression> pathExpression;
     private final List<Mutable<ILogicalExpression>> partitionExpressions;
     private final List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> orderExpressions;
+    private final List<Mutable<ILogicalExpression>> keyExpressions;
+    private final boolean autogenerated;
     private final IWriteDataSink writeDataSink;
 
     public WriteOperator(Mutable<ILogicalExpression> sourceExpression, Mutable<ILogicalExpression> pathExpression,
             List<Mutable<ILogicalExpression>> partitionExpressions,
             List<Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>>> orderExpressions,
-            IWriteDataSink writeDataSink) {
+            List<Mutable<ILogicalExpression>> keyExpressions, boolean autogenerated, IWriteDataSink writeDataSink) {
         this.sourceExpression = sourceExpression;
         this.pathExpression = pathExpression;
         this.partitionExpressions = partitionExpressions;
         this.orderExpressions = orderExpressions;
         this.writeDataSink = writeDataSink;
+        this.keyExpressions = keyExpressions;
+        this.autogenerated = autogenerated;
     }
 
     public Mutable<ILogicalExpression> getSourceExpression() {
@@ -74,6 +78,18 @@
         return orderExpressions;
     }
 
+    public List<Mutable<ILogicalExpression>> getKeyExpressions() {
+        return keyExpressions;
+    }
+
+    public List<LogicalVariable> getKeyVariables() {
+        List<LogicalVariable> keyVariables = new ArrayList<>();
+        for (Mutable<ILogicalExpression> keyExpression : keyExpressions) {
+            keyVariables.add(VariableUtilities.getVariable(keyExpression.getValue()));
+        }
+        return keyVariables;
+    }
+
     public List<LogicalVariable> getPartitionVariables() {
         List<LogicalVariable> partitionVariables = new ArrayList<>();
         for (Mutable<ILogicalExpression> partitionExpression : partitionExpressions) {
@@ -92,10 +108,18 @@
         return orderColumns;
     }
 
+    public boolean getAutogenerated() {
+        return autogenerated;
+    }
+
     public IWriteDataSink getWriteDataSink() {
         return writeDataSink;
     }
 
+    public boolean isFileStoreSink() {
+        return keyExpressions.isEmpty() && !autogenerated;
+    }
+
     @Override
     public LogicalOperatorTag getOperatorTag() {
         return LogicalOperatorTag.WRITE;
@@ -119,6 +143,10 @@
             changed |= visitor.transform(orderExpressionPair.second);
         }
 
+        for (Mutable<ILogicalExpression> expression : keyExpressions) {
+            changed |= visitor.transform(expression);
+        }
+
         return changed;
     }
 
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 fa47ae5..8664acf 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
@@ -297,9 +297,11 @@
                 deepCopyExpressionRefs(new ArrayList<>(), op.getPartitionExpressions());
         List<Pair<IOrder, Mutable<ILogicalExpression>>> newOrderPairExpressions =
                 deepCopyOrderAndExpression(op.getOrderExpressions());
+        List<Mutable<ILogicalExpression>> newKeyPairExpressions =
+                deepCopyExpressionRefs(new ArrayList<>(), op.getKeyExpressions());
         IWriteDataSink writeDataSink = op.getWriteDataSink().createCopy();
         return new WriteOperator(newSourceExpression, newPathExpression, newPartitionExpressions,
-                newOrderPairExpressions, writeDataSink);
+                newOrderPairExpressions, newKeyPairExpressions, op.getAutogenerated(), writeDataSink);
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index d0b0608..50401c3 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -389,6 +389,7 @@
         for (Pair<IOrder, Mutable<ILogicalExpression>> orderExpr : op.getOrderExpressions()) {
             substUsedVariablesInExpr(orderExpr.second, pair.first, pair.second);
         }
+        substUsedVariablesInExpr(op.getKeyExpressions(), pair.first, pair.second);
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index d7b2555..7186f7e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -362,6 +362,10 @@
         for (Pair<IOrder, Mutable<ILogicalExpression>> orderExpr : op.getOrderExpressions()) {
             orderExpr.second.getValue().getUsedVariables(usedVariables);
         }
+
+        for (Mutable<ILogicalExpression> expr : op.getKeyExpressions()) {
+            expr.getValue().getUsedVariables(usedVariables);
+        }
         return null;
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
index d462cd5..71f5876 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
@@ -23,6 +23,7 @@
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.ListSet;
@@ -61,12 +62,16 @@
     private final LogicalVariable sourceVariable;
     private final List<LogicalVariable> partitionVariables;
     private final List<OrderColumn> orderColumns;
+    private final List<LogicalVariable> keyVariables;
+    private final boolean autogenerated;
 
     public SinkWritePOperator(LogicalVariable sourceVariable, List<LogicalVariable> partitionVariables,
-            List<OrderColumn> orderColumns) {
+            List<OrderColumn> orderColumns, List<LogicalVariable> keyVariables, boolean autogenerated) {
         this.sourceVariable = sourceVariable;
         this.partitionVariables = partitionVariables;
         this.orderColumns = orderColumns;
+        this.keyVariables = keyVariables;
+        this.autogenerated = autogenerated;
     }
 
     @Override
@@ -145,6 +150,16 @@
         IBinaryComparatorFactory[] partitionComparatorFactories =
                 JobGenHelper.variablesToAscBinaryComparatorFactories(partitionVariables, typeEnv, context);
 
+        // Key expressions
+        IScalarEvaluatorFactory[] keyEvalFactories = new IScalarEvaluatorFactory[write.getKeyExpressions().size()];
+        List<Mutable<ILogicalExpression>> keyExpressions = write.getKeyExpressions();
+        if (!keyExpressions.isEmpty()) {
+            for (int i = 0; i < keyExpressions.size(); i++) {
+                ILogicalExpression keyExpr = keyExpressions.get(i).getValue();
+                keyEvalFactories[i] = runtimeProvider.createEvaluatorFactory(keyExpr, typeEnv, inputSchemas, context);
+            }
+        }
+
         RecordDescriptor recDesc =
                 JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
         RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(
@@ -152,9 +167,17 @@
 
         IMetadataProvider<?, ?> mp = context.getMetadataProvider();
 
-        Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getWriteFileRuntime(
-                sourceColumn, partitionColumns, partitionComparatorFactories, dynamicPathEvalFactory, staticPathExpr,
-                pathExpr.getSourceLocation(), writeDataSink, inputDesc, typeEnv.getVarType(sourceVariable));
+        Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> runtimeAndConstraints;
+        if (write.isFileStoreSink()) {
+            runtimeAndConstraints = mp.getWriteFileRuntime(sourceColumn, partitionColumns, partitionComparatorFactories,
+                    dynamicPathEvalFactory, staticPathExpr, pathExpr.getSourceLocation(), writeDataSink, inputDesc,
+                    typeEnv.getVarType(sourceVariable));
+
+        } else {
+            runtimeAndConstraints = mp.getWriteDatabaseWithKeyRuntime(sourceColumn, keyEvalFactories, autogenerated,
+                    writeDataSink, inputDesc, typeEnv.getVarType(sourceVariable));
+        }
+
         IPushRuntimeFactory runtime = runtimeAndConstraints.first;
         runtime.setSourceLocation(write.getSourceLocation());
 
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index f945994..3ea2631 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -408,7 +408,9 @@
             }
             ensureAllVariables(op.getPartitionExpressions(), v -> v);
             ensureAllVariables(op.getOrderExpressions(), Pair::getSecond);
-            return new SinkWritePOperator(op.getSourceVariable(), op.getPartitionVariables(), op.getOrderColumns());
+            ensureAllVariables(op.getKeyExpressions(), v -> v);
+            return new SinkWritePOperator(op.getSourceVariable(), op.getPartitionVariables(), op.getOrderColumns(),
+                    op.getKeyVariables(), op.getAutogenerated());
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/IWriterPartitioner.java
similarity index 70%
copy from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
copy to hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/IWriterPartitioner.java
index a4fa97b..cb72aec 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/writer/IExternalFilePrinterFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/IWriterPartitioner.java
@@ -16,16 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.runtime.writer;
+package org.apache.hyracks.algebricks.runtime.operators.writer;
 
-import java.io.Serializable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
-/**
- * {@link IExternalFileWriter} printer factory
- */
-public interface IExternalFilePrinterFactory extends Serializable {
-    /**
-     * @return a new external file printer
-     */
-    IExternalFilePrinter createPrinter();
-}
+interface IWriterPartitioner {
+    boolean isNewPartition(FrameTupleAccessor tupleAccessor, int index) throws HyracksDataException;
+
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/KeyWriterPartitioner.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/KeyWriterPartitioner.java
new file mode 100644
index 0000000..3cb44ff
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/KeyWriterPartitioner.java
@@ -0,0 +1,35 @@
+/*
+ * 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.runtime.operators.writer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+class KeyWriterPartitioner implements IWriterPartitioner {
+    public static final IWriterPartitioner INSTANCE = new KeyWriterPartitioner();
+
+    private KeyWriterPartitioner() {
+    }
+
+    @Override
+    public boolean isNewPartition(FrameTupleAccessor tupleAccessor, int index) throws HyracksDataException {
+        // Every key is a partition
+        return true;
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/NoOpWriterPartitioner.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/NoOpWriterPartitioner.java
new file mode 100644
index 0000000..3221d73
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/NoOpWriterPartitioner.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.algebricks.runtime.operators.writer;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+class NoOpWriterPartitioner implements IWriterPartitioner {
+    private boolean first = true;
+
+    public NoOpWriterPartitioner() {
+    }
+
+    @Override
+    public boolean isNewPartition(FrameTupleAccessor tupleAccessor, int index) throws HyracksDataException {
+        if (first) {
+            first = false;
+            return true;
+        }
+        return false;
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/SinkExternalWriterRuntime.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/SinkExternalWriterRuntime.java
index 01e137b..9407c08 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/SinkExternalWriterRuntime.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/SinkExternalWriterRuntime.java
@@ -23,44 +23,30 @@
 import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputSinkPushRuntime;
 import org.apache.hyracks.algebricks.runtime.writers.IExternalWriter;
 import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-import org.apache.hyracks.dataflow.common.data.accessors.PermutingFrameTupleReference;
-import org.apache.hyracks.dataflow.common.data.accessors.PointableTupleReference;
-import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupWriter;
 
 final class SinkExternalWriterRuntime extends AbstractOneInputSinkPushRuntime {
     private final int sourceColumn;
-    private final int[] partitionColumns;
+    private final IWriterPartitioner partitioner;
     private final IPointable sourceValue;
-    private final PointableTupleReference partitionColumnsPrevCopy;
-    private final PermutingFrameTupleReference partitionColumnsRef;
-    private final IBinaryComparator[] partitionComparators;
     private final IExternalWriter writer;
     private FrameTupleAccessor tupleAccessor;
     private FrameTupleReference tupleRef;
-    private boolean first;
     private IFrameWriter frameWriter;
 
-    SinkExternalWriterRuntime(int sourceColumn, int[] partitionColumns, IBinaryComparator[] partitionComparators,
-            RecordDescriptor inputRecordDesc, IExternalWriter writer) {
+    SinkExternalWriterRuntime(int sourceColumn, IWriterPartitioner partitioner, RecordDescriptor inputRecordDesc,
+            IExternalWriter writer) {
         this.sourceColumn = sourceColumn;
-        this.partitionColumns = partitionColumns;
+        this.partitioner = partitioner;
         this.sourceValue = new VoidPointable();
-        partitionColumnsRef = new PermutingFrameTupleReference(partitionColumns);
-        partitionColumnsPrevCopy =
-                PointableTupleReference.create(partitionColumns.length, ArrayBackedValueStorage::new);
-        this.partitionComparators = partitionComparators;
         this.inputRecordDesc = inputRecordDesc;
         this.writer = writer;
-        first = true;
     }
 
     @Override
@@ -83,8 +69,6 @@
             }
             setValue(tupleRef, sourceColumn, sourceValue);
             writer.write(sourceValue);
-            partitionColumnsRef.reset(tupleAccessor, i);
-            partitionColumnsPrevCopy.set(partitionColumnsRef);
         }
     }
 
@@ -106,13 +90,7 @@
     }
 
     private boolean isNewPartition(int index) throws HyracksDataException {
-        if (first) {
-            first = false;
-            return true;
-        }
-
-        return !PreclusteredGroupWriter.sameGroup(partitionColumnsPrevCopy, tupleAccessor, index, partitionColumns,
-                partitionComparators);
+        return partitioner.isNewPartition(tupleAccessor, index);
     }
 
     private void setValue(IFrameTupleReference tuple, int column, IPointable value) {
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/SinkExternalWriterRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/SinkExternalWriterRuntimeFactory.java
index 6220dec..321828f 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/SinkExternalWriterRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/SinkExternalWriterRuntimeFactory.java
@@ -31,17 +31,30 @@
 public final class SinkExternalWriterRuntimeFactory extends AbstractPushRuntimeFactory {
     private static final long serialVersionUID = -2215789207336628581L;
     private final int sourceColumn;
-    private final int[] partitionColumn;
+    private final int[] partitionColumns;
     private final IBinaryComparatorFactory[] partitionComparatorFactories;
+    private final boolean partitionByKey;
     private final RecordDescriptor inputRecordDescriptor;
     private final IExternalWriterFactory writerFactory;
 
-    public SinkExternalWriterRuntimeFactory(int sourceColumn, int[] partitionColumn,
+    public SinkExternalWriterRuntimeFactory(int sourceColumn, int[] partitionColumns,
             IBinaryComparatorFactory[] partitionComparatorFactories, RecordDescriptor inputRecordDescriptor,
             IExternalWriterFactory writerFactory) {
+        this(sourceColumn, partitionColumns, partitionComparatorFactories, false, inputRecordDescriptor, writerFactory);
+    }
+
+    public SinkExternalWriterRuntimeFactory(int sourceColumn, RecordDescriptor inputRecordDescriptor,
+            IExternalWriterFactory writerFactory) {
+        this(sourceColumn, null, null, true, inputRecordDescriptor, writerFactory);
+    }
+
+    private SinkExternalWriterRuntimeFactory(int sourceColumn, int[] partitionColumns,
+            IBinaryComparatorFactory[] partitionComparatorFactories, boolean partitionByKey,
+            RecordDescriptor inputRecordDescriptor, IExternalWriterFactory writerFactory) {
         this.sourceColumn = sourceColumn;
-        this.partitionColumn = partitionColumn;
+        this.partitionColumns = partitionColumns;
         this.partitionComparatorFactories = partitionComparatorFactories;
+        this.partitionByKey = partitionByKey;
         this.inputRecordDescriptor = inputRecordDescriptor;
         this.writerFactory = writerFactory;
     }
@@ -49,12 +62,33 @@
     @Override
     public IPushRuntime[] createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
         IExternalWriter writer = writerFactory.createWriter(ctx);
-        IBinaryComparator[] partitionComparators = new IBinaryComparator[partitionComparatorFactories.length];
-        for (int i = 0; i < partitionComparatorFactories.length; i++) {
-            partitionComparators[i] = partitionComparatorFactories[i].createBinaryComparator();
-        }
-        SinkExternalWriterRuntime runtime = new SinkExternalWriterRuntime(sourceColumn, partitionColumn,
-                partitionComparators, inputRecordDescriptor, writer);
+        SinkExternalWriterRuntime runtime =
+                new SinkExternalWriterRuntime(sourceColumn, createPartitioner(), inputRecordDescriptor, writer);
         return new IPushRuntime[] { runtime };
     }
+
+    /**
+     * Creates the writer partitioner based on the provided parameters
+     *
+     * @return writer partitioner
+     */
+    private IWriterPartitioner createPartitioner() {
+        // key writer partitioner
+        if (partitionByKey) {
+            return KeyWriterPartitioner.INSTANCE;
+        }
+
+        // writer partitioner
+        if (partitionColumns.length > 0) {
+            IBinaryComparator[] comparators = new IBinaryComparator[partitionComparatorFactories.length];
+            for (int i = 0; i < partitionComparatorFactories.length; i++) {
+                comparators[i] = partitionComparatorFactories[i].createBinaryComparator();
+            }
+
+            return new WriterPartitioner(partitionColumns, comparators);
+        }
+
+        // no-op partitioner
+        return new NoOpWriterPartitioner();
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/WriterPartitioner.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/WriterPartitioner.java
new file mode 100644
index 0000000..468ca4a
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/writer/WriterPartitioner.java
@@ -0,0 +1,62 @@
+/*
+ * 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.runtime.operators.writer;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.data.accessors.PermutingFrameTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.PointableTupleReference;
+import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupWriter;
+
+class WriterPartitioner implements IWriterPartitioner {
+    private final int[] partitionColumns;
+    private final IBinaryComparator[] partitionComparators;
+    private final PointableTupleReference partitionColumnsPrevCopy;
+    private final PermutingFrameTupleReference partitionColumnsRef;
+    private boolean first = true;
+
+    public WriterPartitioner(int[] partitionColumns, IBinaryComparator[] partitionComparators) {
+        this.partitionColumns = partitionColumns;
+        this.partitionComparators = partitionComparators;
+        partitionColumnsRef = new PermutingFrameTupleReference(partitionColumns);
+        partitionColumnsPrevCopy =
+                PointableTupleReference.create(partitionColumns.length, ArrayBackedValueStorage::new);
+    }
+
+    @Override
+    public boolean isNewPartition(FrameTupleAccessor tupleAccessor, int index) throws HyracksDataException {
+        if (first) {
+            first = false;
+            partitionColumnsRef.reset(tupleAccessor, index);
+            partitionColumnsPrevCopy.set(partitionColumnsRef);
+            return true;
+        }
+
+        boolean newPartition = !PreclusteredGroupWriter.sameGroup(partitionColumnsPrevCopy, tupleAccessor, index,
+                partitionColumns, partitionComparators);
+
+        // Set previous
+        partitionColumnsRef.reset(tupleAccessor, index);
+        partitionColumnsPrevCopy.set(partitionColumnsRef);
+
+        return newPartition;
+    }
+}
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index cc739b0..b9a7dbb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -19,7 +19,7 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
   <modelVersion>4.0.0</modelVersion>
   <artifactId>apache-asterixdb-fullstack</artifactId>
-  <version>0.9.9</version>
+  <version>0.9.10-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>hyracks-asterix</name>