You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by na...@apache.org on 2020/07/31 04:21:30 UTC

[hudi] branch master updated: [HUDI-394] Provide a basic implementation of test suite

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 2fc2b01  [HUDI-394] Provide a basic implementation of test suite
2fc2b01 is described below

commit 2fc2b01d8647aafe2862db14fc169aa40d4ae3d5
Author: Nishith Agarwal <na...@uber.com>
AuthorDate: Fri Nov 1 12:40:09 2019 -0700

    [HUDI-394] Provide a basic implementation of test suite
---
 .travis.yml                                        |   2 +-
 .../demo/config/test-suite/base.properties         |  20 +-
 docker/demo/config/test-suite/complex-dag-cow.yaml |  97 ++++
 docker/demo/config/test-suite/complex-dag-mor.yaml | 119 +++++
 docker/demo/config/test-suite/complex-source.avsc  | 466 +++++++++++++++++
 .../demo/config/test-suite/source.avsc             |  60 ++-
 .../demo/config/test-suite/target.avsc             |  63 ++-
 .../demo/config/test-suite/test-source.properties  |  27 +
 docker/hoodie/hadoop/hive_base/Dockerfile          |   1 +
 docker/hoodie/hadoop/hive_base/pom.xml             |   4 +-
 .../common/testutils/HoodieTestDataGenerator.java  |  17 +-
 .../java/org/apache/hudi/hive/HiveSyncTool.java    |   9 +
 .../org/apache/hudi/hive/HoodieHiveClient.java     |   4 +
 .../hudi/hive/testutils/HiveTestService.java       |  15 +-
 hudi-integ-test/README.md                          | 262 ++++++++++
 hudi-integ-test/pom.xml                            | 205 ++++++++
 hudi-integ-test/prepare_integration_suite.sh       | 174 +++++++
 .../testsuite/HoodieDeltaStreamerWrapper.java      |  77 +++
 .../hudi/integ/testsuite/HoodieTestSuiteJob.java   | 186 +++++++
 .../integ/testsuite/HoodieTestSuiteWriter.java     | 219 ++++++++
 .../testsuite/configuration/DFSDeltaConfig.java    |  73 +++
 .../integ/testsuite/configuration/DeltaConfig.java | 256 ++++++++++
 .../hudi/integ/testsuite/converter/Converter.java  |  28 +-
 .../integ/testsuite/converter/UpdateConverter.java |  53 ++
 .../apache/hudi/integ/testsuite/dag/DagUtils.java  | 217 ++++++++
 .../hudi/integ/testsuite/dag/ExecutionContext.java |  54 ++
 .../testsuite/dag/SimpleWorkflowDagGenerator.java  |  75 +++
 .../hudi/integ/testsuite/dag/WorkflowDag.java      |  28 +-
 .../integ/testsuite/dag/WorkflowDagGenerator.java  |  24 +-
 .../integ/testsuite/dag/nodes/BulkInsertNode.java  |  33 +-
 .../hudi/integ/testsuite/dag/nodes/CleanNode.java  |  27 +-
 .../integ/testsuite/dag/nodes/CompactNode.java     |  47 ++
 .../hudi/integ/testsuite/dag/nodes/DagNode.java    | 125 +++++
 .../integ/testsuite/dag/nodes/HiveQueryNode.java   |  87 ++++
 .../integ/testsuite/dag/nodes/HiveSyncNode.java    |  46 ++
 .../hudi/integ/testsuite/dag/nodes/InsertNode.java |  60 +++
 .../integ/testsuite/dag/nodes/RollbackNode.java    |  48 ++
 .../testsuite/dag/nodes/ScheduleCompactNode.java   |  56 ++
 .../testsuite/dag/nodes/SparkSQLQueryNode.java     |  61 +++
 .../hudi/integ/testsuite/dag/nodes/UpsertNode.java |  52 ++
 .../integ/testsuite/dag/nodes/ValidateNode.java    |  48 ++
 .../testsuite/dag/scheduler/DagScheduler.java      | 100 ++++
 .../integ/testsuite/generator/DeltaGenerator.java  | 237 +++++++++
 .../FlexibleSchemaRecordGenerationIterator.java    |  70 +++
 .../GenericRecordFullPayloadGenerator.java         | 285 +++++++++++
 .../GenericRecordFullPayloadSizeEstimator.java     | 121 +++++
 .../GenericRecordPartialPayloadGenerator.java      |  82 +++
 .../generator/LazyRecordGeneratorIterator.java     |  35 +-
 .../generator/UpdateGeneratorIterator.java         |  56 ++
 .../helpers/DFSTestSuitePathSelector.java          |  92 ++++
 .../testsuite/helpers/HiveServiceProvider.java     |  74 +++
 .../testsuite/reader/DFSAvroDeltaInputReader.java  |  89 ++++
 .../testsuite/reader/DFSDeltaInputReader.java      |  98 ++++
 .../reader/DFSHoodieDatasetInputReader.java        | 338 ++++++++++++
 .../integ/testsuite/reader/DeltaInputReader.java   |  62 +++
 .../integ/testsuite/reader/DeltaInputType.java     |  25 +-
 .../integ/testsuite/reader/SparkBasedReader.java   |  70 +++
 .../testsuite/writer/AvroFileDeltaInputWriter.java | 117 +++++
 .../testsuite/writer/DFSDeltaWriterAdapter.java    |  62 +++
 .../testsuite/writer/DFSSparkAvroDeltaWriter.java  |  29 +-
 .../integ/testsuite/writer/DeltaInputWriter.java   |  42 +-
 .../integ/testsuite/writer/DeltaOutputMode.java    |  25 +-
 .../integ/testsuite/writer/DeltaWriteStats.java    |  72 +++
 .../integ/testsuite/writer/DeltaWriterAdapter.java |  27 +-
 .../integ/testsuite/writer/DeltaWriterFactory.java |  57 +++
 .../writer/SparkAvroDeltaInputWriter.java          |  69 +++
 .../TestDFSHoodieTestSuiteWriterAdapter.java       | 138 +++++
 .../integ/testsuite/TestFileDeltaInputWriter.java  | 147 ++++++
 .../configuration/TestWorkflowBuilder.java         |  80 +++
 .../testsuite/converter/TestUpdateConverter.java   |  93 ++++
 .../integ/testsuite/dag/ComplexDagGenerator.java   |  81 +++
 .../integ/testsuite/dag/HiveSyncDagGenerator.java  |  58 +++
 .../testsuite/dag/HiveSyncDagGeneratorMOR.java     |  58 +++
 .../hudi/integ/testsuite/dag/TestDagUtils.java     |  95 ++++
 .../TestGenericRecordPayloadEstimator.java         |  56 ++
 .../TestGenericRecordPayloadGenerator.java         | 130 +++++
 .../testsuite/job/TestHoodieTestSuiteJob.java      | 224 ++++++++
 .../reader/TestDFSAvroDeltaInputReader.java        |  74 +++
 .../reader/TestDFSHoodieDatasetInputReader.java    | 119 +++++
 .../hudi/integ/testsuite/utils/TestUtils.java      |  63 +++
 .../test/resources/log4j-surefire-quiet.properties |   7 +-
 .../main/java/org/apache/hudi/QuickstartUtils.java |   4 +-
 .../apache/hudi/keygen/ComplexKeyGenerator.java    |  13 +-
 .../java/org/apache/hudi/keygen/KeyGenerator.java  |   2 +-
 .../hudi/keygen/NonpartitionedKeyGenerator.java    |   2 +-
 .../org/apache/hudi/keygen/SimpleKeyGenerator.java |  10 +-
 .../hudi/keygen/TimestampBasedKeyGenerator.java    |   1 -
 .../src/test/java/TestComplexKeyGenerator.java     |  67 +++
 hudi-utilities/pom.xml                             |  10 +
 .../org/apache/hudi/utilities/UtilHelpers.java     |  13 +
 .../hudi/utilities/deltastreamer/DeltaSync.java    |  57 ++-
 .../deltastreamer/HoodieDeltaStreamer.java         |  20 +-
 .../deltastreamer/HoodieDeltaStreamerMetrics.java  |   8 +-
 .../deltastreamer/SourceFormatAdapter.java         |   8 +-
 .../hudi/utilities/sources/AvroDFSSource.java      |   9 +-
 .../utilities/sources/helpers/DFSPathSelector.java |  16 +-
 .../hudi/utilities/sources/TestCsvDFSSource.java   |   2 +-
 .../utilities/testutils/UtilitiesTestBase.java     |  35 +-
 .../delta-streamer-config/complex-source.avsc      | 466 +++++++++++++++++
 packaging/hudi-integ-test-bundle/pom.xml           | 565 +++++++++++++++++++++
 .../org/apache/hudi/testsuite/bundle/Main.java     |  29 +-
 pom.xml                                            |   5 +-
 102 files changed, 8281 insertions(+), 343 deletions(-)

diff --git a/.travis.yml b/.travis.yml
index 1454c6d..67fa007 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -27,7 +27,7 @@ jobs:
     - name: "Functional tests"
       env: MODE=functional HUDI_QUIETER_LOGGING=1
     - name: "Integration tests"
-      env: MODE=integration
+      env: MODE=integration HUDI_QUIETER_LOGGING=1
 install: true
 services:
   - docker
diff --git a/hudi-integ-test/src/test/resources/log4j-surefire-quiet.properties b/docker/demo/config/test-suite/base.properties
similarity index 58%
copy from hudi-integ-test/src/test/resources/log4j-surefire-quiet.properties
copy to docker/demo/config/test-suite/base.properties
index b21b5d4..13b1acc 100644
--- a/hudi-integ-test/src/test/resources/log4j-surefire-quiet.properties
+++ b/docker/demo/config/test-suite/base.properties
@@ -1,4 +1,4 @@
-###
+#
 # 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
@@ -14,16 +14,8 @@
 # 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.
-###
-log4j.rootLogger=WARN, CONSOLE
-log4j.logger.org.apache.hudi=DEBUG
-
-# CONSOLE is set to be a ConsoleAppender.
-log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
-# CONSOLE uses PatternLayout.
-log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
-log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
-log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
-log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
-log4j.appender.CONSOLE.filter.a.LevelMin=WARN
-log4j.appender.CONSOLE.filter.a.LevelMax=FATAL
+#
+hoodie.upsert.shuffle.parallelism=2
+hoodie.insert.shuffle.parallelism=2
+hoodie.bulkinsert.shuffle.parallelism=2
+hoodie.datasource.write.partitionpath.field=timestamp
diff --git a/docker/demo/config/test-suite/complex-dag-cow.yaml b/docker/demo/config/test-suite/complex-dag-cow.yaml
new file mode 100644
index 0000000..5a97688
--- /dev/null
+++ b/docker/demo/config/test-suite/complex-dag-cow.yaml
@@ -0,0 +1,97 @@
+# 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.
+first_insert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    repeat_count: 5
+    num_records_insert: 1000
+  type: InsertNode
+  deps: none
+second_insert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    repeat_count: 5
+    num_records_insert: 10000
+  deps: first_insert
+  type: InsertNode
+third_insert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    repeat_count: 2
+    num_records_insert: 300
+  deps: second_insert
+  type: InsertNode
+first_rollback:
+  config:
+  deps: third_insert
+  type: RollbackNode
+first_upsert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    num_records_insert: 300
+    repeat_count: 5
+    num_records_upsert: 100
+    num_upsert_partitions: 10
+  type: UpsertNode
+  deps: first_rollback
+first_hive_sync:
+  config:
+    queue_name: "adhoc"
+    engine: "mr"
+  type: HiveSyncNode
+  deps: first_upsert
+first_hive_query:
+  config:
+    hive_props:
+      prop1: "set hive.execution.engine=spark"
+      prop2: "set spark.yarn.queue="
+      prop3: "set hive.strict.checks.large.query=false"
+      prop4: "set hive.stats.autogather=false"
+    hive_queries:
+      query1: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1"
+      result1: 0
+      query2: "select count(*) from testdb1.table1"
+      result2: 22100000
+  type: HiveQueryNode
+  deps: first_hive_sync
+second_upsert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    num_records_insert: 300
+    repeat_count: 5
+    num_records_upsert: 100
+    num_upsert_partitions: 10
+  type: UpsertNode
+  deps: first_hive_query
+second_hive_query:
+  config:
+    hive_props:
+      prop1: "set hive.execution.engine=mr"
+      prop2: "set mapred.job.queue.name="
+      prop3: "set hive.strict.checks.large.query=false"
+      prop4: "set hive.stats.autogather=false"
+    hive_queries:
+      query1: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1"
+      result1: 0
+      query2: "select count(*) from testdb1.table1"
+      result2: 22100
+  type: HiveQueryNode
+  deps: second_upsert
\ No newline at end of file
diff --git a/docker/demo/config/test-suite/complex-dag-mor.yaml b/docker/demo/config/test-suite/complex-dag-mor.yaml
new file mode 100644
index 0000000..3981603
--- /dev/null
+++ b/docker/demo/config/test-suite/complex-dag-mor.yaml
@@ -0,0 +1,119 @@
+# 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.
+first_insert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    repeat_count: 5
+    num_records_insert: 100
+  type: InsertNode
+  deps: none
+second_insert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    repeat_count: 5
+    num_records_insert: 100
+  deps: first_insert
+  type: InsertNode
+third_insert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    repeat_count: 2
+    num_records_insert: 300
+  deps: second_insert
+  type: InsertNode
+first_rollback:
+  config:
+  deps: third_insert
+  type: RollbackNode
+first_upsert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    num_records_insert: 300
+    repeat_count: 5
+    num_records_upsert: 100
+    num_upsert_partitions: 10
+  type: UpsertNode
+  deps: first_rollback
+first_hive_sync:
+  config:
+    queue_name: "adhoc"
+    engine: "mr"
+  type: HiveSyncNode
+  deps: first_upsert
+first_hive_query:
+  config:
+    queue_name: "adhoc"
+    engine: "mr"
+  type: HiveQueryNode
+  deps: first_hive_sync
+second_upsert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    num_records_insert: 300
+    repeat_count: 5
+    num_records_upsert: 100
+    num_upsert_partitions: 10
+  type: UpsertNode
+  deps: first_hive_query
+second_hive_query:
+  config:
+    queue_name: "adhoc"
+    engine: "mr"
+    hive_queries:
+      query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1"
+      result1: 0
+      query2: "select count(*) from testdb.table1"
+      result2: 3100
+      query3: "select count(*) from testdb.table1_rt group by `_row_key` having count(*) > 1"
+      result3: 0
+      query4: "select count(*) from testdb.table1_rt"
+      result4: 3100
+  type: HiveQueryNode
+  deps: second_upsert
+first_schedule_compact:
+  config:
+  type: ScheduleCompactNode
+  deps: second_hive_query
+third_upsert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    num_records_insert: 300
+    repeat_count: 5
+    num_records_upsert: 100
+    num_upsert_partitions: 10
+  type: UpsertNode
+  deps: first_schedule_compact
+first_compact:
+  config:
+  type: CompactNode
+  deps: first_schedule_compact
+third_hive_query:
+  config:
+    queue_name: "adhoc"
+    engine: "mr"
+    hive_queries:
+      query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1"
+      result1: 0
+      query2: "select count(*) from testdb.table1"
+      result2: 2210
+  type: HiveQueryNode
+  deps: second_upsert
\ No newline at end of file
diff --git a/docker/demo/config/test-suite/complex-source.avsc b/docker/demo/config/test-suite/complex-source.avsc
new file mode 100644
index 0000000..a202d2d
--- /dev/null
+++ b/docker/demo/config/test-suite/complex-source.avsc
@@ -0,0 +1,466 @@
+/*
+ * 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.
+ */
+{
+    "name": "COMPLEX",
+    "fields": [
+        {
+            "default": null,
+            "type": [
+                "null",
+                {
+                    "items": "string",
+                    "type": "array"
+                }
+            ],
+            "name": "array_of_string_fields1"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field1"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field2"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field3"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field4"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field5"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "boolean"
+            ],
+            "name": "boolean_field1"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field6"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field7"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field8"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field9"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field10"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field11"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field12"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "double"
+            ],
+            "name": "double_field1"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field13"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field1"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field14"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field2"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                {
+                    "items": {
+                        "fields": [
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "string"
+                                ],
+                                "name": "string_field15"
+                            },
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "string"
+                                ],
+                                "name": "string_field16"
+                            },
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "string"
+                                ],
+                                "name": "string_field17"
+                            },
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "long"
+                                ],
+                                "name": "long_field3"
+                            },
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "long"
+                                ],
+                                "name": "long_field4"
+                            },
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "double"
+                                ],
+                                "name": "double_field2"
+                            },
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "double"
+                                ],
+                                "name": "double_field3"
+                            }
+                        ],
+                        "type": "record",
+                        "name": "record_field1"
+                    },
+                    "type": "array"
+                }
+            ],
+            "name": "record_name1"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field18"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field5"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "double"
+            ],
+            "name": "double_field4"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "double"
+            ],
+            "name": "double_field5"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field19"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field6"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field20"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field7"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "double"
+            ],
+            "name": "double_field6"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field21"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field22"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field23"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field8"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "double"
+            ],
+            "name": "double_field7"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field24"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field10"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field25"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field26"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field11"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "boolean"
+            ],
+            "name": "boolean_field3"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field12"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "double"
+            ],
+            "name": "double_field8"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field13"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field27"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field28"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field29"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field30"
+        }
+    ],
+    "type": "record"
+}
\ No newline at end of file
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/docker/demo/config/test-suite/source.avsc
similarity index 52%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to docker/demo/config/test-suite/source.avsc
index 7dde326..c2a8d7c 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/docker/demo/config/test-suite/source.avsc
@@ -7,7 +7,7 @@
  * "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
+ *     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,
@@ -15,29 +15,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.hudi.keygen;
-
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
-
-import org.apache.avro.generic.GenericRecord;
-
-import java.io.Serializable;
-
-/**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
- */
-public abstract class KeyGenerator implements Serializable {
-
-  protected transient TypedProperties config;
-
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
-  }
-
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
+{
+  "type" : "record",
+  "name" : "triprec",
+  "fields" : [
+  {
+    "name" : "timestamp",
+    "type" : "long"
+  }, {
+    "name" : "_row_key",
+    "type" : "string"
+  }, {
+    "name" : "rider",
+    "type" : "string"
+  }, {
+    "name" : "driver",
+    "type" : "string"
+  }, {
+    "name" : "begin_lat",
+    "type" : "double"
+  }, {
+    "name" : "begin_lon",
+    "type" : "double"
+  }, {
+    "name" : "end_lat",
+    "type" : "double"
+  }, {
+    "name" : "end_lon",
+    "type" : "double"
+  }, {
+    "name" : "fare",
+    "type" : "double"
+  } ]
 }
+
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/docker/demo/config/test-suite/target.avsc
similarity index 52%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to docker/demo/config/test-suite/target.avsc
index 7dde326..11e23a4 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/docker/demo/config/test-suite/target.avsc
@@ -7,7 +7,7 @@
  * "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
+ *     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,
@@ -15,29 +15,40 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-package org.apache.hudi.keygen;
-
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
-
-import org.apache.avro.generic.GenericRecord;
-
-import java.io.Serializable;
-
-/**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
- */
-public abstract class KeyGenerator implements Serializable {
-
-  protected transient TypedProperties config;
-
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
-  }
-
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
+{
+  "type" : "record",
+  "name" : "triprec",
+  "fields" : [
+  {
+    "name" : "timestamp",
+    "type" : "double"
+  }, {
+    "name" : "_row_key",
+    "type" : "string"
+  }, {
+    "name" : "rider",
+    "type" : "string"
+  }, {
+    "name" : "driver",
+    "type" : "string"
+  }, {
+    "name" : "begin_lat",
+    "type" : "double"
+  }, {
+    "name" : "begin_lon",
+    "type" : "double"
+  }, {
+    "name" : "end_lat",
+    "type" : "double"
+  }, {
+    "name" : "end_lon",
+    "type" : "double"
+  }, {
+    "name" : "fare",
+    "type" : "double"
+  }, {
+    "name" : "haversine_distance",
+    "type" : "double"
+  }]
 }
+
diff --git a/docker/demo/config/test-suite/test-source.properties b/docker/demo/config/test-suite/test-source.properties
new file mode 100644
index 0000000..397f871
--- /dev/null
+++ b/docker/demo/config/test-suite/test-source.properties
@@ -0,0 +1,27 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+hoodie.datasource.write.recordkey.field=_row_key
+hoodie.deltastreamer.source.dfs.root=/user/hive/warehouse/hudi-bench/input
+hoodie.datasource.write.keygenerator.class=org.apache.hudi.ComplexKeyGenerator
+hoodie.datasource.write.partitionpath.field=timestamp
+hoodie.deltastreamer.schemaprovider.source.schema.file=/var/hoodie/ws/docker/demo/config/bench/source.avsc
+hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000/
+hoodie.datasource.hive_sync.database=testdb
+hoodie.datasource.hive_sync.table=test_table
+hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.NonPartitionedExtractor
+hoodie.datasource.hive_sync.assume_date_partitioning=true
+hoodie.datasource.write.keytranslator.class=org.apache.hudi.DayBasedPartitionPathKeyTranslator
+hoodie.deltastreamer.schemaprovider.target.schema.file=/var/hoodie/ws/docker/demo/config/bench/source.avsc
\ No newline at end of file
diff --git a/docker/hoodie/hadoop/hive_base/Dockerfile b/docker/hoodie/hadoop/hive_base/Dockerfile
index b8452d6..8d85fd5 100644
--- a/docker/hoodie/hadoop/hive_base/Dockerfile
+++ b/docker/hoodie/hadoop/hive_base/Dockerfile
@@ -55,6 +55,7 @@ ENV HUDI_HADOOP_BUNDLE=/var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/hood
 ENV HUDI_HIVE_SYNC_BUNDLE=/var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/hoodie-hive-sync-bundle.jar
 ENV HUDI_SPARK_BUNDLE=/var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/hoodie-spark-bundle.jar
 ENV HUDI_UTILITIES_BUNDLE=/var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/hoodie-utilities.jar
+ENV HUDI_INTEG_TEST_BUNDLE=/var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/hoodie-integ-test-bundle.jar
 
 COPY startup.sh /usr/local/bin/
 RUN chmod +x /usr/local/bin/startup.sh
diff --git a/docker/hoodie/hadoop/hive_base/pom.xml b/docker/hoodie/hadoop/hive_base/pom.xml
index 1a632eb..399f7b7 100644
--- a/docker/hoodie/hadoop/hive_base/pom.xml
+++ b/docker/hoodie/hadoop/hive_base/pom.xml
@@ -59,9 +59,7 @@
                 <copy file="${project.basedir}/../../../../packaging/hudi-hadoop-mr-bundle/target/hudi-hadoop-mr-bundle-${project.version}.jar" tofile="target/hoodie-hadoop-mr-bundle.jar" />
                 <copy file="${project.basedir}/../../../../packaging/hudi-hive-sync-bundle/target/hudi-hive-sync-bundle-${project.version}.jar" tofile="target/hoodie-hive-sync-bundle.jar" />
                 <copy file="${project.basedir}/../../../../packaging/hudi-spark-bundle/target/hudi-spark-bundle_${scala.binary.version}-${project.version}.jar" tofile="target/hoodie-spark-bundle.jar" />
-                <copy
-                  file="${project.basedir}/../../../../packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_${scala.binary.version}-${project.version}.jar"
-                  tofile="target/hoodie-utilities.jar"/>
+                <copy file="${project.basedir}/../../../../packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_${scala.binary.version}-${project.version}.jar" tofile="target/hoodie-utilities.jar"/>
               </tasks>
             </configuration>
             <goals>
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java
index e23d885..ed7a458 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java
@@ -68,7 +68,6 @@ import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import java.util.stream.Stream;
 
-
 /**
  * Class to be used in tests to keep generating test inserts and updates against a corpus.
  * <p>
@@ -98,7 +97,8 @@ public class HoodieTestDataGenerator {
       + "{\"name\": \"amount\",\"type\": \"double\"},{\"name\": \"currency\", \"type\": \"string\"}]}},";
   public static final String FARE_FLATTENED_SCHEMA = "{\"name\": \"fare\", \"type\": \"double\"},"
       + "{\"name\": \"currency\", \"type\": \"string\"},";
-  public static final String TIP_NESTED_SCHEMA = "{\"name\": \"tip_history\", \"type\": {\"type\": \"array\", \"items\": {\"type\": \"record\", \"name\": \"tip_history\", \"fields\": ["
+  public static final String TIP_NESTED_SCHEMA = "{\"name\": \"tip_history\", \"default\": null, \"type\": {\"type\": "
+      + "\"array\", \"items\": {\"type\": \"record\", \"default\": null, \"name\": \"tip_history\", \"fields\": ["
       + "{\"name\": \"amount\", \"type\": \"double\"}, {\"name\": \"currency\", \"type\": \"string\"}]}}},";
   public static final String MAP_TYPE_SCHEMA = "{\"name\": \"city_to_state\", \"type\": {\"type\": \"map\", \"values\": \"string\"}},";
   public static final String EXTRA_TYPE_SCHEMA = "{\"name\": \"distance_in_meters\", \"type\": \"int\"},"
@@ -254,7 +254,6 @@ public class HoodieTestDataGenerator {
     rec.put("begin_lon", RAND.nextDouble());
     rec.put("end_lat", RAND.nextDouble());
     rec.put("end_lon", RAND.nextDouble());
-
     if (isFlattened) {
       rec.put("fare", RAND.nextDouble() * 100);
       rec.put("currency", "USD");
@@ -730,7 +729,7 @@ public class HoodieTestDataGenerator {
   public boolean deleteExistingKeyIfPresent(HoodieKey key) {
     Map<Integer, KeyPartition> existingKeys = existingKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
     Integer numExistingKeys = numKeysBySchema.get(TRIP_EXAMPLE_SCHEMA);
-    for (Map.Entry<Integer, KeyPartition> entry: existingKeys.entrySet()) {
+    for (Map.Entry<Integer, KeyPartition> entry : existingKeys.entrySet()) {
       if (entry.getValue().key.equals(key)) {
         int index = entry.getKey();
         existingKeys.put(index, existingKeys.get(numExistingKeys - 1));
@@ -740,10 +739,18 @@ public class HoodieTestDataGenerator {
         return true;
       }
     }
-
     return false;
   }
 
+  public List<GenericRecord> generateGenericRecords(int numRecords) {
+    List<GenericRecord> list = new ArrayList<>();
+    IntStream.range(0, numRecords).forEach(i -> {
+      list.add(generateGenericRecord(UUID.randomUUID().toString(), UUID.randomUUID().toString(), UUID.randomUUID()
+          .toString(), RAND.nextDouble()));
+    });
+    return list;
+  }
+
   public String[] getPartitionPaths() {
     return partitionPaths;
   }
diff --git a/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java b/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java
index 877ba47..d829252 100644
--- a/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java
+++ b/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java
@@ -112,6 +112,15 @@ public class HiveSyncTool {
         + " of type " + hoodieHiveClient.getTableType());
     // Check if the necessary table exists
     boolean tableExists = hoodieHiveClient.doesTableExist(tableName);
+
+    // check if the database exists else create it
+    try {
+      hoodieHiveClient.updateHiveSQL("create database if not exists " + cfg.databaseName);
+    } catch (Exception e) {
+      // this is harmless since table creation will fail anyways, creation of DB is needed for in-memory testing
+      LOG.warn("Unable to create database", e);
+    }
+
     // Get the parquet schema for this table looking at the latest commit
     MessageType schema = hoodieHiveClient.getDataSchema();
     // Sync schema if needed
diff --git a/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java b/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java
index f1034e3..2bb3fd1 100644
--- a/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java
+++ b/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java
@@ -572,4 +572,8 @@ public class HoodieHiveClient {
       return new PartitionEvent(PartitionEventType.UPDATE, storagePartition);
     }
   }
+
+  public IMetaStoreClient getClient() {
+    return client;
+  }
 }
\ No newline at end of file
diff --git a/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java b/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java
index 2f98803..51a5bf5 100644
--- a/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java
+++ b/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestService.java
@@ -126,10 +126,21 @@ public class HiveTestService {
   public void stop() {
     resetSystemProperties();
     if (tServer != null) {
-      tServer.stop();
+      try {
+        tServer.stop();
+      } catch (Exception e) {
+        LOG.error("Stop meta store failed", e);
+      }
     }
     if (hiveServer != null) {
-      hiveServer.stop();
+      try {
+        hiveServer.stop();
+      } catch (Exception e) {
+        LOG.error("Stop hive server failed", e);
+      }
+    }
+    if (executorService != null) {
+      executorService.shutdownNow();
     }
     LOG.info("Hive Minicluster service shut down.");
     tServer = null;
diff --git a/hudi-integ-test/README.md b/hudi-integ-test/README.md
new file mode 100644
index 0000000..d87fec3
--- /dev/null
+++ b/hudi-integ-test/README.md
@@ -0,0 +1,262 @@
+<!--
+  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.
+-->
+
+This page describes in detail how to run end to end tests on a hudi dataset that helps in improving our confidence 
+in a release as well as perform large scale performance benchmarks.  
+
+# Objectives
+
+1. Test with different versions of core libraries and components such as `hdfs`, `parquet`, `spark`, 
+`hive` and `avro`.
+2. Generate different types of workloads across different dimensions such as `payload size`, `number of updates`, 
+`number of inserts`, `number of partitions`
+3. Perform multiple types of operations such as `insert`, `bulk_insert`, `upsert`, `compact`, `query`
+4. Support custom post process actions and validations
+
+# High Level Design
+
+The Hudi test suite runs as a long running spark job. The suite is divided into the following high level components : 
+
+## Workload Generation
+
+This component does the work of generating the workload; `inserts`, `upserts` etc.
+
+## Workload Scheduling
+
+Depending on the type of workload generated, data is either ingested into the target hudi 
+dataset or the corresponding workload operation is executed. For example compaction does not necessarily need a workload
+to be generated/ingested but can require an execution.
+
+## Other actions/operatons
+
+The test suite supports different types of operations besides ingestion such as Hive Query execution, Clean action etc.
+
+# Usage instructions
+
+
+## Entry class to the test suite
+
+```
+org.apache.hudi.testsuite.HoodieTestSuiteJob.java - Entry Point of the hudi test suite job. This 
+class wraps all the functionalities required to run a configurable integration suite.
+```
+
+## Configurations required to run the job
+```
+org.apache.hudi.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig - Config class that drives the behavior of the 
+integration test suite. This class extends from com.uber.hoodie.utilities.DeltaStreamerConfig. Look at 
+link#HudiDeltaStreamer page to learn about all the available configs applicable to your test suite.
+```
+
+## Generating a custom Workload Pattern
+
+There are 2 ways to generate a workload pattern
+
+ 1.Programatically
+
+Choose to write up the entire DAG of operations programatically, take a look at `WorkflowDagGenerator` class.
+Once you're ready with the DAG you want to execute, simply pass the class name as follows:
+
+```
+spark-submit
+...
+...
+--class org.apache.hudi.testsuite.HoodieTestSuiteJob 
+--workload-generator-classname org.apache.hudi.testsuite.dag.scheduler.<your_workflowdaggenerator>
+...
+```
+
+ 2.YAML file
+
+Choose to write up the entire DAG of operations in YAML, take a look at `complex-workload-dag-cow.yaml` or 
+`complex-workload-dag-mor.yaml`.
+Once you're ready with the DAG you want to execute, simply pass the yaml file path as follows:
+
+```
+spark-submit
+...
+...
+--class org.apache.hudi.testsuite.HoodieTestSuiteJob 
+--workload-yaml-path /path/to/your-workflow-dag.yaml
+...
+```
+
+## Building the test suite
+
+The test suite can be found in the `hudi-integ-test` module. Use the `prepare_integration_suite.sh` script to 
+build 
+the test suite, you can provide different parameters to the script.
+
+```
+shell$ ./prepare_integration_suite.sh --help
+Usage: prepare_integration_suite.sh
+   --spark-command, prints the spark command
+   -h, hdfs-version
+   -s, spark version
+   -p, parquet version
+   -a, avro version
+   -s, hive version
+```
+
+```
+shell$ ./prepare_integration_suite.sh
+....
+....
+Final command : mvn clean install -DskipTests
+```
+
+## Running on the cluster or in your local machine
+Copy over the necessary files and jars that are required to your cluster and then run the following spark-submit 
+command after replacing the correct values for the parameters. 
+NOTE : The properties-file should have all the necessary information required to ingest into a Hudi dataset. For more
+ information on what properties need to be set, take a look at the test suite section under demo steps.
+```
+shell$ ./prepare_integration_suite.sh --spark-command
+spark-submit --packages com.databricks:spark-avro_2.11:4.0.0 --master prepare_integration_suite.sh --deploy-mode
+--properties-file  --class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob target/hudi-integ-test-0.6
+.0-SNAPSHOT.jar --source-class  --source-ordering-field  --input-base-path  --target-base-path  --target-table  --props  --storage-type  --payload-class  --workload-yaml-path  --input-file-size  --<deltastreamer-ingest>
+```
+
+## Running through a test-case (local)
+Take a look at the `TestHoodieTestSuiteJob` to check how you can run the entire suite using JUnit.
+
+## Running an end to end test suite in Local Docker environment
+
+```
+docker exec -it adhoc-2 /bin/bash
+# COPY_ON_WRITE tables
+=========================
+## Run the following command to start the test suite
+spark-submit \ 
+--packages com.databricks:spark-avro_2.11:4.0.0 \ 
+--conf spark.task.cpus=1 \ 
+--conf spark.executor.cores=1 \ 
+--conf spark.task.maxFailures=100 \ 
+--conf spark.memory.fraction=0.4 \ 
+--conf spark.rdd.compress=true \ 
+--conf spark.kryoserializer.buffer.max=2000m \ 
+--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ 
+--conf spark.memory.storageFraction=0.1 \ 
+--conf spark.shuffle.service.enabled=true \ 
+--conf spark.sql.hive.convertMetastoreParquet=false \ 
+--conf spark.ui.port=5555 \ 
+--conf spark.driver.maxResultSize=12g \ 
+--conf spark.executor.heartbeatInterval=120s \ 
+--conf spark.network.timeout=600s \ 
+--conf spark.eventLog.overwrite=true \ 
+--conf spark.eventLog.enabled=true \ 
+--conf spark.yarn.max.executor.failures=10 \ 
+--conf spark.sql.catalogImplementation=hive \ 
+--conf spark.sql.shuffle.partitions=1000 \ 
+--class org.apache.hudi.testsuite.HoodieTestSuiteJob $HUDI_TEST_SUITE_BUNDLE \
+--source-ordering-field timestamp \ 
+--target-base-path /user/hive/warehouse/hudi-integ-test-suite/output \ 
+--input-base-path /user/hive/warehouse/hudi-integ-test-suite/input \ 
+--target-table test_table \ 
+--props /var/hoodie/ws/docker/demo/config/testsuite/test-source.properties \ 
+--schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider \ 
+--source-limit 300000 \ 
+--source-class org.apache.hudi.utilities.sources.AvroDFSSource \ 
+--input-file-size 125829120 \ 
+--workload-yaml-path /var/hoodie/ws/docker/demo/config/testsuite/complex-workflow-dag-cow.yaml \ 
+--storage-type COPY_ON_WRITE \ 
+--compact-scheduling-minshare 1 \ 
+--hoodie-conf "hoodie.deltastreamer.source.test.num_partitions=100" \ 
+--hoodie-conf "hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false" \ 
+--hoodie-conf "hoodie.deltastreamer.source.test.max_unique_records=100000000" \ 
+--hoodie-conf "hoodie.embed.timeline.server=false" \ 
+--hoodie-conf "hoodie.datasource.write.recordkey.field=_row_key" \ 
+--hoodie-conf "hoodie.deltastreamer.source.dfs.root=/user/hive/warehouse/hudi-integ-test-suite/input" \ 
+--hoodie-conf "hoodie.datasource.write.keygenerator.class=org.apache.hudi.ComplexKeyGenerator" \ 
+--hoodie-conf "hoodie.datasource.write.partitionpath.field=timestamp" \ 
+--hoodie-conf "hoodie.deltastreamer.schemaprovider.source.schema.file=/var/hoodie/ws/docker/demo/config/testsuite/source.avsc" \ 
+--hoodie-conf "hoodie.datasource.hive_sync.assume_date_partitioning=false" \ 
+--hoodie-conf "hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000/" \ 
+--hoodie-conf "hoodie.datasource.hive_sync.database=testdb" \ 
+--hoodie-conf "hoodie.datasource.hive_sync.table=test_table" \ 
+--hoodie-conf "hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.NonPartitionedExtractor" \ 
+--hoodie-conf "hoodie.datasource.hive_sync.assume_date_partitioning=true" \ 
+--hoodie-conf "hoodie.datasource.write.keytranslator.class=org.apache.hudi.DayBasedPartitionPathKeyTranslator" \ 
+--hoodie-conf "hoodie.deltastreamer.schemaprovider.target.schema.file=/var/hoodie/ws/docker/demo/config/testsuite/source.avsc"
+...
+...
+2019-11-03 05:44:47 INFO  DagScheduler:69 - ----------- Finished workloads ----------
+2019-11-03 05:44:47 INFO  HoodieTestSuiteJob:138 - Finished scheduling all tasks
+...
+2019-11-03 05:44:48 INFO  SparkContext:54 - Successfully stopped SparkContext
+# MERGE_ON_READ tables
+=========================
+## Run the following command to start the test suite
+spark-submit \ 
+--packages com.databricks:spark-avro_2.11:4.0.0 \ 
+--conf spark.task.cpus=1 \ 
+--conf spark.executor.cores=1 \ 
+--conf spark.task.maxFailures=100 \ 
+--conf spark.memory.fraction=0.4 \ 
+--conf spark.rdd.compress=true \ 
+--conf spark.kryoserializer.buffer.max=2000m \ 
+--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \ 
+--conf spark.memory.storageFraction=0.1 \ 
+--conf spark.shuffle.service.enabled=true \ 
+--conf spark.sql.hive.convertMetastoreParquet=false \ 
+--conf spark.ui.port=5555 \ 
+--conf spark.driver.maxResultSize=12g \ 
+--conf spark.executor.heartbeatInterval=120s \ 
+--conf spark.network.timeout=600s \ 
+--conf spark.eventLog.overwrite=true \ 
+--conf spark.eventLog.enabled=true \ 
+--conf spark.yarn.max.executor.failures=10 \ 
+--conf spark.sql.catalogImplementation=hive \ 
+--conf spark.sql.shuffle.partitions=1000 \ 
+--class org.apache.hudi.testsuite.HoodieTestSuiteJob $HUDI_TEST_SUITE_BUNDLE \
+--source-ordering-field timestamp \ 
+--target-base-path /user/hive/warehouse/hudi-integ-test-suite/output \ 
+--input-base-path /user/hive/warehouse/hudi-integ-test-suite/input \ 
+--target-table test_table \ 
+--props /var/hoodie/ws/docker/demo/config/testsuite/test-source.properties \ 
+--schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider \ 
+--source-limit 300000 \ 
+--source-class org.apache.hudi.utilities.sources.AvroDFSSource \ 
+--input-file-size 125829120 \ 
+--workload-yaml-path /var/hoodie/ws/docker/demo/config/testsuite/complex-workflow-dag-mor.yaml \ 
+--storage-type MERGE_ON_READ \ 
+--compact-scheduling-minshare 1 \ 
+--hoodie-conf "hoodie.deltastreamer.source.test.num_partitions=100" \ 
+--hoodie-conf "hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false" \ 
+--hoodie-conf "hoodie.deltastreamer.source.test.max_unique_records=100000000" \ 
+--hoodie-conf "hoodie.embed.timeline.server=false" \ 
+--hoodie-conf "hoodie.datasource.write.recordkey.field=_row_key" \ 
+--hoodie-conf "hoodie.deltastreamer.source.dfs.root=/user/hive/warehouse/hudi-integ-test-suite/input" \ 
+--hoodie-conf "hoodie.datasource.write.keygenerator.class=org.apache.hudi.ComplexKeyGenerator" \ 
+--hoodie-conf "hoodie.datasource.write.partitionpath.field=timestamp" \ 
+--hoodie-conf "hoodie.deltastreamer.schemaprovider.source.schema.file=/var/hoodie/ws/docker/demo/config/testsuite/source.avsc" \ 
+--hoodie-conf "hoodie.datasource.hive_sync.assume_date_partitioning=false" \ 
+--hoodie-conf "hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000/" \ 
+--hoodie-conf "hoodie.datasource.hive_sync.database=testdb" \ 
+--hoodie-conf "hoodie.datasource.hive_sync.table=test_table" \ 
+--hoodie-conf "hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.NonPartitionedExtractor" \ 
+--hoodie-conf "hoodie.datasource.hive_sync.assume_date_partitioning=true" \ 
+--hoodie-conf "hoodie.datasource.write.keytranslator.class=org.apache.hudi.DayBasedPartitionPathKeyTranslator" \ 
+--hoodie-conf "hoodie.deltastreamer.schemaprovider.target.schema.file=/var/hoodie/ws/docker/demo/config/testsuite/source.avsc"
+...
+...
+2019-11-03 05:44:47 INFO  DagScheduler:69 - ----------- Finished workloads ----------
+2019-11-03 05:44:47 INFO  HoodieTestSuiteJob:138 - Finished scheduling all tasks
+...
+2019-11-03 05:44:48 INFO  SparkContext:54 - Successfully stopped SparkContext
+```
+ 
\ No newline at end of file
diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml
index b9bcc6d..e81f335 100644
--- a/hudi-integ-test/pom.xml
+++ b/hudi-integ-test/pom.xml
@@ -51,12 +51,69 @@
       <scope>test</scope>
     </dependency>
 
+    <!-- Jetty -->
+    <dependency>
+      <!-- Needs to be at the top to ensure we get the correct dependency versions for jetty-server -->
+      <groupId>org.eclipse.jetty.aggregate</groupId>
+      <artifactId>jetty-all</artifactId>
+      <version>${jetty.version}</version>
+      <classifier>uber</classifier>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet.jsp</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
     <!-- Hoodie -->
     <dependency>
       <groupId>org.apache.hudi</groupId>
       <artifactId>hudi-spark_${scala.binary.version}</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-utilities_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet.jsp</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.databricks</groupId>
+          <artifactId>spark-avro_${scala.binary.version}</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
 
     <!-- Logging -->
     <dependency>
@@ -89,6 +146,67 @@
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-hive-sync</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet.jsp</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+      <classifier>tests</classifier>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-utilities_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <type>test-jar</type>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet.jsp</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-client</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+    </dependency>
+
+    <!-- Fasterxml -->
+    <dependency>
+      <groupId>com.fasterxml.jackson.dataformat</groupId>
+      <artifactId>jackson-dataformat-yaml</artifactId>
+      <version>2.7.4</version>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+      <version>2.6.7.3</version>
+    </dependency>
 
     <!-- Fasterxml - Test-->
     <dependency>
@@ -107,6 +225,87 @@
       <scope>test</scope>
     </dependency>
 
+    <!-- HDFS test dependencies -->
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <classifier>tests</classifier>
+      <exclusions>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet.jsp</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <classifier>tests</classifier>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <artifactId>netty</artifactId>
+          <groupId>io.netty</groupId>
+        </exclusion>
+        <exclusion>
+          <artifactId>netty-all</artifactId>
+          <groupId>io.netty</groupId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- Hive dependencies -->
+    <dependency>
+      <groupId>${hive.groupid}</groupId>
+      <artifactId>hive-exec</artifactId>
+      <version>${hive.version}</version>
+      <classifier>${hive.exec.classifier}</classifier>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>${hive.groupid}</groupId>
+      <artifactId>hive-jdbc</artifactId>
+      <version>${hive.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet.jsp</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.eclipse.jetty</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+      <scope>test</scope>
+    </dependency>
+
     <dependency>
       <groupId>org.awaitility</groupId>
       <artifactId>awaitility</artifactId>
@@ -138,6 +337,12 @@
       <scope>test</scope>
     </dependency>
 
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-junit-jupiter</artifactId>
+      <scope>test</scope>
+    </dependency>
+
   </dependencies>
 
   <properties>
diff --git a/hudi-integ-test/prepare_integration_suite.sh b/hudi-integ-test/prepare_integration_suite.sh
new file mode 100644
index 0000000..06999d9
--- /dev/null
+++ b/hudi-integ-test/prepare_integration_suite.sh
@@ -0,0 +1,174 @@
+# 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.
+#!/bin/bash
+
+# Determine the current working directory
+_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
+# Preserve the calling directory
+_CALLING_DIR="$(pwd)"
+
+#########################
+# The command line help #
+#########################
+usage() {
+    echo "Usage: $0"
+    echo "   --spark-command, prints the spark command"
+    echo "   -h | --hadoop, hadoop-version"
+    echo "   -s | --spark, spark version"
+    echo "   -p | --parquet, parquet version"
+    echo "   -a | --avro, avro version"
+    echo "   -i | --hive, hive version"
+    echo "   -l | --scala, scala version"
+    exit 1
+}
+
+get_spark_command() {
+if [ -z "$scala" ]
+then
+  scala="2.11"
+else
+  scala=$1
+fi
+echo "spark-submit --packages org.apache.spark:spark-avro_${scala}:2.4.4 \
+--master $0 \
+--deploy-mode $1 \
+--properties-file $2 \
+--class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob \
+`ls target/hudi-integ-test-*-SNAPSHOT.jar` \
+--source-class $3 \
+--source-ordering-field $4 \
+--input-base-path $5 \
+--target-base-path $6 \
+--target-table $7 \
+--props $8 \
+--storage-type $9 \
+--payload-class "${10}" \
+--workload-yaml-path "${11}" \
+--input-file-size "${12}" \
+--<use-deltastreamer>"
+}
+
+case "$1" in
+   --help)
+       usage
+       exit 0
+       ;;
+esac
+
+case "$1" in
+   --spark-command)
+       get_spark_command
+       exit 0
+       ;;
+esac
+
+while getopts ":h:s:p:a:i:l:-:" opt; do
+  case $opt in
+    h) hadoop="$OPTARG"
+    printf "Argument hadoop is %s\n" "$hadoop"
+    ;;
+    s) spark="$OPTARG"
+    printf "Argument spark is %s\n" "$spark"
+    ;;
+    p) parquet="$OPTARG"
+    printf "Argument parquet is %s\n" "$parquet"
+    ;;
+    a) avro="$OPTARG"
+    printf "Argument avro is %s\n" "$avro"
+    ;;
+    i) hive="$OPTARG"
+    printf "Argument hive is %s\n" "$hive"
+    ;;
+    l) scala="$OPTARG"
+    printf "Argument scala is %s\n" "$scala"
+    ;;
+    -)
+      case "$OPTARG" in
+        hadoop)
+          hadoop="${!OPTIND}"; OPTIND=$(( $OPTIND + 1 ))
+          printf "Argument hadoop is %s\n" "$hadoop"
+          ;;
+        spark)
+          spark="${!OPTIND}"; OPTIND=$(( $OPTIND + 1 ))
+          printf "Argument spark is %s\n" "$spark"
+          ;;
+        parquet)
+          parquet="${!OPTIND}"; OPTIND=$(( $OPTIND + 1 ))
+          printf "Argument parquet is %s\n" "$parquet"
+          ;;
+        avro)
+          avro="${!OPTIND}"; OPTIND=$(( $OPTIND + 1 ))
+          printf "Argument avro is %s\n" "$avro"
+          ;;
+        hive)
+          hive="${!OPTIND}"; OPTIND=$(( $OPTIND + 1 ))
+          printf "Argument hive is %s\n" "$hive"
+          ;;
+        scala)
+          scala="${!OPTIND}"; OPTIND=$(( $OPTIND + 1 ))
+          printf "Argument scala is %s\n" "$scala"
+          ;;
+        *) echo "Invalid option --$OPTARG" >&2
+          ;;
+    esac ;;
+    \?) echo "Invalid option -$OPTARG" >&2
+    ;;
+  esac
+done
+
+
+get_versions () {
+  base_command=''
+  if [ -z "$hadoop" ]
+   then
+    base_command=$base_command
+  else
+    hadoop=$1
+    base_command+=' -Dhadoop.version='$hadoop
+  fi
+
+  if [ -z "$hive" ]
+  then
+    base_command=$base_command
+  else
+    hive=$2
+    base_command+=' -Dhive.version='$hive
+  fi
+
+  if [ -z "$scala" ]
+  then
+    base_command=$base_command
+  else
+    scala=$3
+    base_command+=' -Dscala-'$scala
+  fi
+  echo $base_command
+}
+
+versions=$(get_versions $hadoop $hive $scala)
+
+final_command='mvn clean install -DskipTests '$versions
+printf "Final command $final_command \n"
+
+# change to the project root directory to run maven command
+move_to_root='cd ..'
+$move_to_root && $final_command
+
+# change back to original working directory
+cd $_CALLING_DIR
+
+printf "A sample spark command to start the integration suite \n"
+get_spark_command $scala
\ No newline at end of file
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java
new file mode 100644
index 0000000..a489bac
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieDeltaStreamerWrapper.java
@@ -0,0 +1,77 @@
+/*
+ * 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.hudi.integ.testsuite;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+
+/**
+ * Extends the {@link HoodieDeltaStreamer} to expose certain operations helpful in running the Test Suite.
+ * This is done to achieve 2 things 1) Leverage some components of {@link HoodieDeltaStreamer} 2)
+ * Piggyback on the suite to test {@link HoodieDeltaStreamer}
+ */
+public class HoodieDeltaStreamerWrapper extends HoodieDeltaStreamer {
+
+  public HoodieDeltaStreamerWrapper(Config cfg, JavaSparkContext jssc) throws Exception {
+    super(cfg, jssc);
+  }
+
+  public HoodieDeltaStreamerWrapper(Config cfg, JavaSparkContext jssc, FileSystem fs, HiveConf conf) throws Exception {
+    super(cfg, jssc, fs, conf);
+  }
+
+  public JavaRDD<WriteStatus> upsert(Operation operation) throws
+      Exception {
+    cfg.operation = operation;
+    return deltaSyncService.getDeltaSync().syncOnce().getRight();
+  }
+
+  public JavaRDD<WriteStatus> insert() throws Exception {
+    return upsert(Operation.INSERT);
+  }
+
+  public JavaRDD<WriteStatus> bulkInsert() throws
+      Exception {
+    return upsert(Operation.BULK_INSERT);
+  }
+
+  public void scheduleCompact() throws Exception {
+    // Since we don't support scheduleCompact() operation in delta-streamer, assume upsert without any data that will
+    // trigger scheduling compaction
+    upsert(Operation.UPSERT);
+  }
+
+  public JavaRDD<WriteStatus> compact() throws Exception {
+    // Since we don't support compact() operation in delta-streamer, assume upsert without any data that will trigger
+    // inline compaction
+    return upsert(Operation.UPSERT);
+  }
+
+  public Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> fetchSource() throws Exception {
+    return deltaSyncService.getDeltaSync().readFromSource(deltaSyncService.getDeltaSync().getCommitTimelineOpt());
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java
new file mode 100644
index 0000000..be036c1
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java
@@ -0,0 +1,186 @@
+/*
+ * 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.hudi.integ.testsuite;
+
+import com.beust.jcommander.JCommander;
+import com.beust.jcommander.Parameter;
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hudi.DataSourceUtils;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig;
+import org.apache.hudi.integ.testsuite.dag.DagUtils;
+import org.apache.hudi.integ.testsuite.dag.WorkflowDag;
+import org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator;
+import org.apache.hudi.integ.testsuite.dag.scheduler.DagScheduler;
+import org.apache.hudi.integ.testsuite.generator.DeltaGenerator;
+import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
+import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is the entry point for running a Hudi Test Suite. Although this class has similarities with
+ * {@link HoodieDeltaStreamer} this class does not extend it since do not want to create a dependency on the changes in
+ * DeltaStreamer.
+ */
+public class HoodieTestSuiteJob {
+
+  private static volatile Logger log = LoggerFactory.getLogger(HoodieTestSuiteJob.class);
+
+  private final HoodieTestSuiteConfig cfg;
+  /**
+   * Bag of properties with source, hoodie client, key generator etc.
+   */
+  TypedProperties props;
+  /**
+   * Schema provider that supplies the command for writing out the generated payloads.
+   */
+  private transient SchemaProvider schemaProvider;
+  /**
+   * Filesystem used.
+   */
+  private transient FileSystem fs;
+  /**
+   * Spark context.
+   */
+  private transient JavaSparkContext jsc;
+  /**
+   * Spark Session.
+   */
+  private transient SparkSession sparkSession;
+  /**
+   * Hive Config.
+   */
+  private transient HiveConf hiveConf;
+
+  private KeyGenerator keyGenerator;
+
+  public HoodieTestSuiteJob(HoodieTestSuiteConfig cfg, JavaSparkContext jsc) throws IOException {
+    this.cfg = cfg;
+    this.jsc = jsc;
+    this.sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate();
+    this.fs = FSUtils.getFs(cfg.inputBasePath, jsc.hadoopConfiguration());
+    this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
+    log.info("Creating workload generator with configs : {}", props.toString());
+    this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jsc);
+    this.hiveConf = getDefaultHiveConf(jsc.hadoopConfiguration());
+    this.keyGenerator = DataSourceUtils.createKeyGenerator(props);
+    if (!fs.exists(new Path(cfg.targetBasePath))) {
+      HoodieTableMetaClient.initTableType(jsc.hadoopConfiguration(), cfg.targetBasePath,
+          HoodieTableType.valueOf(cfg.tableType), cfg.targetTableName, "archived");
+    }
+  }
+
+  private static HiveConf getDefaultHiveConf(Configuration cfg) {
+    HiveConf hiveConf = new HiveConf();
+    hiveConf.addResource(cfg);
+    return hiveConf;
+  }
+
+  public static void main(String[] args) throws Exception {
+    final HoodieTestSuiteConfig cfg = new HoodieTestSuiteConfig();
+    JCommander cmd = new JCommander(cfg, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+
+    JavaSparkContext jssc = UtilHelpers.buildSparkContext("workload-generator-" + cfg.outputTypeName
+        + "-" + cfg.inputFormatName, cfg.sparkMaster);
+    new HoodieTestSuiteJob(cfg, jssc).runTestSuite();
+  }
+
+  public void runTestSuite() {
+    try {
+      WorkflowDag workflowDag = this.cfg.workloadYamlPath == null ? ((WorkflowDagGenerator) ReflectionUtils
+          .loadClass((this.cfg).workloadDagGenerator)).build()
+          : DagUtils.convertYamlPathToDag(this.fs, this.cfg.workloadYamlPath);
+      log.info("Workflow Dag => " + DagUtils.convertDagToYaml(workflowDag));
+      long startTime = System.currentTimeMillis();
+      String schemaStr = schemaProvider.getSourceSchema().toString();
+      final HoodieTestSuiteWriter writer = new HoodieTestSuiteWriter(jsc, props, cfg, schemaStr);
+      final DeltaGenerator deltaGenerator = new DeltaGenerator(
+          new DFSDeltaConfig(DeltaOutputMode.valueOf(cfg.outputTypeName), DeltaInputType.valueOf(cfg.inputFormatName),
+              new SerializableConfiguration(jsc.hadoopConfiguration()), cfg.inputBasePath, cfg.targetBasePath,
+              schemaStr, cfg.limitFileSize), jsc, sparkSession, schemaStr, keyGenerator);
+      DagScheduler dagScheduler = new DagScheduler(workflowDag, writer, deltaGenerator);
+      dagScheduler.schedule();
+      log.info("Finished scheduling all tasks, Time taken {}", System.currentTimeMillis() - startTime);
+    } catch (Exception e) {
+      log.error("Failed to run Test Suite ", e);
+      throw new HoodieException("Failed to run Test Suite ", e);
+    } finally {
+      jsc.stop();
+    }
+  }
+
+  /**
+   * The Hudi test suite uses {@link HoodieDeltaStreamer} to run some operations hence extend delta streamer config.
+   */
+  public static class HoodieTestSuiteConfig extends HoodieDeltaStreamer.Config {
+
+    @Parameter(names = {"--input-base-path"}, description = "base path for input data"
+        + "(Will be created if did not exist first time around. If exists, more data will be added to that path)",
+        required = true)
+    public String inputBasePath;
+
+    @Parameter(names = {
+        "--workload-generator-classname"}, description = "WorkflowDag of operations to generate the workload",
+        required = true)
+    public String workloadDagGenerator = WorkflowDagGenerator.class.getName();
+
+    @Parameter(names = {
+        "--workload-yaml-path"}, description = "Workflow Dag yaml path to generate the workload")
+    public String workloadYamlPath;
+
+    @Parameter(names = {"--delta-output-type"}, description = "Subclass of "
+        + "org.apache.hudi.testsuite.workload.DeltaOutputMode to readAvro data.")
+    public String outputTypeName = DeltaOutputMode.DFS.name();
+
+    @Parameter(names = {"--delta-input-format"}, description = "Subclass of "
+        + "org.apache.hudi.testsuite.workload.DeltaOutputMode to read avro data.")
+    public String inputFormatName = DeltaInputType.AVRO.name();
+
+    @Parameter(names = {"--input-file-size"}, description = "The min/max size of the input files to generate",
+        required = true)
+    public Long limitFileSize = 1024 * 1024 * 120L;
+
+    @Parameter(names = {"--use-deltastreamer"}, description = "Choose whether to use HoodieDeltaStreamer to "
+        + "perform"
+        + " ingestion. If set to false, HoodieWriteClient will be used")
+    public Boolean useDeltaStreamer = false;
+
+  }
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java
new file mode 100644
index 0000000..b22faca
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.integ.testsuite;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.client.HoodieReadClient;
+import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieCompactionConfig;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.integ.testsuite.dag.nodes.CleanNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.RollbackNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.ScheduleCompactNode;
+import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Operation;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+
+/**
+ * A writer abstraction for the Hudi test suite. This class wraps different implementations of writers used to perform
+ * write operations into the target hudi dataset. Current supported writers are {@link HoodieDeltaStreamerWrapper}
+ * and {@link HoodieWriteClient}.
+ */
+public class HoodieTestSuiteWriter {
+
+  private HoodieDeltaStreamerWrapper deltaStreamerWrapper;
+  private HoodieWriteClient writeClient;
+  protected HoodieTestSuiteConfig cfg;
+  private Option<String> lastCheckpoint;
+  private HoodieReadClient hoodieReadClient;
+  private Properties props;
+  private String schema;
+  private transient Configuration configuration;
+  private transient JavaSparkContext sparkContext;
+  private static Set<String> VALID_DAG_NODES_TO_ALLOW_WRITE_CLIENT_IN_DELTASTREAMER_MODE = new HashSet<>(
+      Arrays.asList(RollbackNode.class.getName(), CleanNode.class.getName(), ScheduleCompactNode.class.getName()));
+
+  public HoodieTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteConfig cfg, String schema) throws
+      Exception {
+    this(jsc, props, cfg, schema, true);
+  }
+
+  public HoodieTestSuiteWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteConfig cfg, String schema,
+      boolean rollbackInflight) throws Exception {
+    // We ensure that only 1 instance of HoodieWriteClient is instantiated for a HoodieTestSuiteWriter
+    // This does not instantiate a HoodieWriteClient until a
+    // {@link HoodieDeltaStreamer#commit(HoodieWriteClient, JavaRDD, Option)} is invoked.
+    this.deltaStreamerWrapper = new HoodieDeltaStreamerWrapper(cfg, jsc);
+    this.hoodieReadClient = new HoodieReadClient(jsc, cfg.targetBasePath);
+    if (!cfg.useDeltaStreamer) {
+      this.writeClient = new HoodieWriteClient(jsc, getHoodieClientConfig(cfg, props, schema), rollbackInflight);
+    }
+    this.cfg = cfg;
+    this.configuration = jsc.hadoopConfiguration();
+    this.sparkContext = jsc;
+    this.props = props;
+    this.schema = schema;
+  }
+
+  private HoodieWriteConfig getHoodieClientConfig(HoodieTestSuiteConfig cfg, Properties props, String schema) {
+    HoodieWriteConfig.Builder builder =
+        HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.targetBasePath)
+            .withAutoCommit(false)
+            .withCompactionConfig(HoodieCompactionConfig.newBuilder().withPayloadClass(cfg.payloadClassName).build())
+            .forTable(cfg.targetTableName)
+            .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
+            .withProps(props);
+    builder = builder.withSchema(schema);
+    return builder.build();
+  }
+
+  private boolean allowWriteClientAccess(DagNode dagNode) {
+    if (VALID_DAG_NODES_TO_ALLOW_WRITE_CLIENT_IN_DELTASTREAMER_MODE.contains(dagNode.getClass().getName())) {
+      return true;
+    }
+    return false;
+  }
+
+  public Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> fetchSource() throws Exception {
+    return this.deltaStreamerWrapper.fetchSource();
+  }
+
+  public Option<String> startCommit() {
+    if (cfg.useDeltaStreamer) {
+      return Option.of(HoodieActiveTimeline.createNewInstantTime());
+    } else {
+      return Option.of(writeClient.startCommit());
+    }
+  }
+
+  public JavaRDD<WriteStatus> upsert(Option<String> instantTime) throws Exception {
+    if (cfg.useDeltaStreamer) {
+      return deltaStreamerWrapper.upsert(Operation.UPSERT);
+    } else {
+      Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> nextBatch = fetchSource();
+      lastCheckpoint = Option.of(nextBatch.getValue().getLeft());
+      return writeClient.upsert(nextBatch.getRight().getRight(), instantTime.get());
+    }
+  }
+
+  public JavaRDD<WriteStatus> insert(Option<String> instantTime) throws Exception {
+    if (cfg.useDeltaStreamer) {
+      return deltaStreamerWrapper.insert();
+    } else {
+      Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> nextBatch = fetchSource();
+      lastCheckpoint = Option.of(nextBatch.getValue().getLeft());
+      return writeClient.insert(nextBatch.getRight().getRight(), instantTime.get());
+    }
+  }
+
+  public JavaRDD<WriteStatus> bulkInsert(Option<String> instantTime) throws Exception {
+    if (cfg.useDeltaStreamer) {
+      return deltaStreamerWrapper.bulkInsert();
+    } else {
+      Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> nextBatch = fetchSource();
+      lastCheckpoint = Option.of(nextBatch.getValue().getLeft());
+      return writeClient.bulkInsert(nextBatch.getRight().getRight(), instantTime.get());
+    }
+  }
+
+  public JavaRDD<WriteStatus> compact(Option<String> instantTime) throws Exception {
+    if (cfg.useDeltaStreamer) {
+      return deltaStreamerWrapper.compact();
+    } else {
+      if (!instantTime.isPresent()) {
+        Option<Pair<String, HoodieCompactionPlan>> compactionPlanPair = Option
+            .fromJavaOptional(hoodieReadClient.getPendingCompactions()
+                .stream().findFirst());
+        if (compactionPlanPair.isPresent()) {
+          instantTime = Option.of(compactionPlanPair.get().getLeft());
+        }
+      }
+      if (instantTime.isPresent()) {
+        return writeClient.compact(instantTime.get());
+      } else {
+        return null;
+      }
+    }
+  }
+
+  public Option<String> scheduleCompaction(Option<Map<String, String>> previousCommitExtraMetadata) throws
+      Exception {
+    if (!cfg.useDeltaStreamer) {
+      deltaStreamerWrapper.scheduleCompact();
+      return Option.empty();
+    } else {
+      return writeClient.scheduleCompaction(previousCommitExtraMetadata);
+    }
+  }
+
+  public void commit(JavaRDD<WriteStatus> records, Option<String> instantTime) {
+    if (!cfg.useDeltaStreamer) {
+      Map<String, String> extraMetadata = new HashMap<>();
+      /** Store the checkpoint in the commit metadata just like
+       * {@link HoodieDeltaStreamer#commit(HoodieWriteClient, JavaRDD, Option)} **/
+      extraMetadata.put(HoodieDeltaStreamerWrapper.CHECKPOINT_KEY, lastCheckpoint.get());
+      writeClient.commit(instantTime.get(), records, Option.of(extraMetadata));
+    }
+  }
+
+  public HoodieWriteClient getWriteClient(DagNode dagNode) throws IllegalAccessException {
+    if (cfg.useDeltaStreamer & !allowWriteClientAccess(dagNode)) {
+      throw new IllegalAccessException("cannot access write client when testing in deltastreamer mode");
+    }
+    synchronized (this) {
+      if (writeClient == null) {
+        this.writeClient = new HoodieWriteClient(this.sparkContext, getHoodieClientConfig(cfg, props, schema), false);
+      }
+    }
+    return writeClient;
+  }
+
+  public HoodieDeltaStreamerWrapper getDeltaStreamerWrapper() {
+    return deltaStreamerWrapper;
+  }
+
+  public HoodieTestSuiteConfig getCfg() {
+    return cfg;
+  }
+
+  public Configuration getConfiguration() {
+    return configuration;
+  }
+
+  public JavaSparkContext getSparkContext() {
+    return sparkContext;
+  }
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DFSDeltaConfig.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DFSDeltaConfig.java
new file mode 100644
index 0000000..2915628
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DFSDeltaConfig.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.integ.testsuite.configuration;
+
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
+import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
+
+/**
+ * Configuration to hold details about a DFS based output type, implements {@link DeltaConfig}.
+ */
+public class DFSDeltaConfig extends DeltaConfig {
+
+  // The base path where the generated data should be written to. This data will in turn be used to write into a hudi
+  // dataset
+  private final String deltaBasePath;
+  private final String datasetOutputPath;
+  private final String schemaStr;
+  // Maximum file size for the files generated
+  private final Long maxFileSize;
+  // The current batch id
+  private Integer batchId;
+
+  public DFSDeltaConfig(DeltaOutputMode deltaOutputMode, DeltaInputType deltaInputType,
+                        SerializableConfiguration configuration,
+                        String deltaBasePath, String targetBasePath, String schemaStr, Long maxFileSize) {
+    super(deltaOutputMode, deltaInputType, configuration);
+    this.deltaBasePath = deltaBasePath;
+    this.schemaStr = schemaStr;
+    this.maxFileSize = maxFileSize;
+    this.datasetOutputPath = targetBasePath;
+  }
+
+  public String getDeltaBasePath() {
+    return deltaBasePath;
+  }
+
+  public String getDatasetOutputPath() {
+    return datasetOutputPath;
+  }
+
+  public String getSchemaStr() {
+    return schemaStr;
+  }
+
+  public Long getMaxFileSize() {
+    return maxFileSize;
+  }
+
+  public Integer getBatchId() {
+    return batchId;
+  }
+
+  public void setBatchId(Integer batchId) {
+    this.batchId = batchId;
+  }
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java
new file mode 100644
index 0000000..30fa584
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/configuration/DeltaConfig.java
@@ -0,0 +1,256 @@
+/*
+ * 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.hudi.integ.testsuite.configuration;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
+import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
+
+/**
+ * Configuration to hold the delta output type and delta input format.
+ */
+public class DeltaConfig implements Serializable {
+
+  private final DeltaOutputMode deltaOutputMode;
+  private final DeltaInputType deltaInputType;
+  private final SerializableConfiguration configuration;
+
+  public DeltaConfig(DeltaOutputMode deltaOutputMode, DeltaInputType deltaInputType,
+      SerializableConfiguration configuration) {
+    this.deltaOutputMode = deltaOutputMode;
+    this.deltaInputType = deltaInputType;
+    this.configuration = configuration;
+  }
+
+  public DeltaOutputMode getDeltaOutputMode() {
+    return deltaOutputMode;
+  }
+
+  public DeltaInputType getDeltaInputType() {
+    return deltaInputType;
+  }
+
+  public Configuration getConfiguration() {
+    return configuration.get();
+  }
+
+  /**
+   * Represents any kind of workload operation for new data. Each workload also contains a set of Option sequence of
+   * actions that can be executed in parallel.
+   */
+  public static class Config {
+
+    public static final String CONFIG_NAME = "config";
+    public static final String TYPE = "type";
+    public static final String NODE_NAME = "name";
+    public static final String DEPENDENCIES = "deps";
+    public static final String CHILDREN = "children";
+    public static final String HIVE_QUERIES = "hive_queries";
+    public static final String HIVE_PROPERTIES = "hive_props";
+    private static String NUM_RECORDS_INSERT = "num_records_insert";
+    private static String NUM_RECORDS_UPSERT = "num_records_upsert";
+    private static String REPEAT_COUNT = "repeat_count";
+    private static String RECORD_SIZE = "record_size";
+    private static String NUM_PARTITIONS_INSERT = "num_partitions_insert";
+    private static String NUM_PARTITIONS_UPSERT = "num_partitions_upsert";
+    private static String NUM_FILES_UPSERT = "num_files_upsert";
+    private static String FRACTION_UPSERT_PER_FILE = "fraction_upsert_per_file";
+    private static String DISABLE_GENERATE = "disable_generate";
+    private static String DISABLE_INGEST = "disable_ingest";
+    private static String HIVE_LOCAL = "hive_local";
+
+    private Map<String, Object> configsMap;
+
+    public Config(Map<String, Object> configsMap) {
+      this.configsMap = configsMap;
+    }
+
+    public static Builder newBuilder() {
+      return new Builder();
+    }
+
+    public long getNumRecordsInsert() {
+      return Long.valueOf(configsMap.getOrDefault(NUM_RECORDS_INSERT, 0).toString());
+    }
+
+    public long getNumRecordsUpsert() {
+      return Long.valueOf(configsMap.getOrDefault(NUM_RECORDS_UPSERT, 0).toString());
+    }
+
+    public int getRecordSize() {
+      return Integer.valueOf(configsMap.getOrDefault(RECORD_SIZE, 1024).toString());
+    }
+
+    public int getNumInsertPartitions() {
+      return Integer.valueOf(configsMap.getOrDefault(NUM_PARTITIONS_INSERT, 1).toString());
+    }
+
+    public int getRepeatCount() {
+      return Integer.valueOf(configsMap.getOrDefault(REPEAT_COUNT, 1).toString());
+    }
+
+    public int getNumUpsertPartitions() {
+      return Integer.valueOf(configsMap.getOrDefault(NUM_PARTITIONS_UPSERT, 0).toString());
+    }
+
+    public int getNumUpsertFiles() {
+      return Integer.valueOf(configsMap.getOrDefault(NUM_FILES_UPSERT, 1).toString());
+    }
+
+    public double getFractionUpsertPerFile() {
+      return Double.valueOf(configsMap.getOrDefault(FRACTION_UPSERT_PER_FILE, 0.0).toString());
+    }
+
+    public boolean isDisableGenerate() {
+      return Boolean.valueOf(configsMap.getOrDefault(DISABLE_GENERATE, false).toString());
+    }
+
+    public boolean isDisableIngest() {
+      return Boolean.valueOf(configsMap.getOrDefault(DISABLE_INGEST, false).toString());
+    }
+
+    public Map<String, Object> getOtherConfigs() {
+      if (configsMap == null) {
+        return new HashMap<>();
+      }
+      return configsMap;
+    }
+
+    public List<Pair<String, Integer>> getHiveQueries() {
+      try {
+        return (List<Pair<String, Integer>>) this.configsMap.getOrDefault("hive_queries", new ArrayList<>());
+      } catch (Exception e) {
+        throw new RuntimeException("unable to get hive queries from configs");
+      }
+    }
+
+    public boolean isHiveLocal() {
+      return Boolean.valueOf(configsMap.getOrDefault(HIVE_LOCAL, false).toString());
+    }
+
+    public List<String> getHiveProperties() {
+      return (List<String>) this.configsMap.getOrDefault(HIVE_PROPERTIES, new ArrayList<>());
+    }
+
+    @Override
+    public String toString() {
+      try {
+        return new ObjectMapper().writeValueAsString(this.configsMap);
+      } catch (Exception e) {
+        throw new RuntimeException("unable to generate string representation of config");
+      }
+    }
+
+    public static class Builder {
+
+      private Map<String, Object> configsMap = new HashMap<>();
+
+      public Builder() {
+      }
+
+      public Builder withNumRecordsToInsert(long numRecordsInsert) {
+        this.configsMap.put(NUM_RECORDS_INSERT, numRecordsInsert);
+        return this;
+      }
+
+      public Builder withNumRecordsToUpdate(long numRecordsUpsert) {
+        this.configsMap.put(NUM_RECORDS_UPSERT, numRecordsUpsert);
+        return this;
+      }
+
+      public Builder withNumInsertPartitions(int numInsertPartitions) {
+        this.configsMap.put(NUM_PARTITIONS_INSERT, numInsertPartitions);
+        return this;
+      }
+
+      public Builder withNumUpsertPartitions(int numUpsertPartitions) {
+        this.configsMap.put(NUM_PARTITIONS_UPSERT, numUpsertPartitions);
+        return this;
+      }
+
+      public Builder withNumUpsertFiles(int numUpsertFiles) {
+        this.configsMap.put(NUM_FILES_UPSERT, numUpsertFiles);
+        return this;
+      }
+
+      public Builder withFractionUpsertPerFile(double fractionUpsertPerFile) {
+        this.configsMap.put(FRACTION_UPSERT_PER_FILE, fractionUpsertPerFile);
+        return this;
+      }
+
+      public Builder withNumTimesToRepeat(int repeatCount) {
+        this.configsMap.put(REPEAT_COUNT, repeatCount);
+        return this;
+      }
+
+      public Builder withRecordSize(int recordSize) {
+        this.configsMap.put(RECORD_SIZE, recordSize);
+        return this;
+      }
+
+      public Builder disableGenerate(boolean generate) {
+        this.configsMap.put(DISABLE_GENERATE, generate);
+        return this;
+      }
+
+      public Builder disableIngest(boolean ingest) {
+        this.configsMap.put(DISABLE_INGEST, ingest);
+        return this;
+      }
+
+      public Builder withConfig(String name, Object value) {
+        this.configsMap.put(name, value);
+        return this;
+      }
+
+      public Builder withHiveQueryAndResults(List<Pair<String, Integer>> hiveQueries) {
+        this.configsMap.put(HIVE_QUERIES, hiveQueries);
+        return this;
+      }
+
+      public Builder withHiveLocal(boolean startHiveLocal) {
+        this.configsMap.put(HIVE_LOCAL, startHiveLocal);
+        return this;
+      }
+
+      public Builder withHiveProperties(List<String> hiveProperties) {
+        this.configsMap.put(HIVE_PROPERTIES, hiveProperties);
+        return this;
+      }
+
+      public Builder withConfigsMap(Map<String, Object> configsMap) {
+        this.configsMap = configsMap;
+        return this;
+      }
+
+      public Config build() {
+        return new Config(configsMap);
+      }
+
+    }
+  }
+}
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/Converter.java
similarity index 58%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/Converter.java
index 7dde326..e4ad0a7 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/Converter.java
@@ -16,28 +16,18 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
-
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
-
-import org.apache.avro.generic.GenericRecord;
+package org.apache.hudi.integ.testsuite.converter;
 
 import java.io.Serializable;
+import org.apache.spark.api.java.JavaRDD;
 
 /**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
+ * Implementations of {@link Converter} will convert data from one format to another.
+ *
+ * @param <I> Input Data Type
+ * @param <O> Output Data Type
  */
-public abstract class KeyGenerator implements Serializable {
-
-  protected transient TypedProperties config;
-
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
-  }
+public interface Converter<I, O> extends Serializable {
 
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
-}
+  JavaRDD<O> convert(JavaRDD<I> inputRDD);
+}
\ No newline at end of file
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/UpdateConverter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/UpdateConverter.java
new file mode 100644
index 0000000..24520a3
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/converter/UpdateConverter.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.integ.testsuite.converter;
+
+import java.util.List;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.integ.testsuite.generator.LazyRecordGeneratorIterator;
+import org.apache.hudi.integ.testsuite.generator.UpdateGeneratorIterator;
+import org.apache.spark.api.java.JavaRDD;
+
+/**
+ * This converter creates an update {@link GenericRecord} from an existing {@link GenericRecord}.
+ */
+public class UpdateConverter implements Converter<GenericRecord, GenericRecord> {
+
+  private final String schemaStr;
+  // The fields that should not be mutated when converting the insert record to an update record, generally the
+  // record_key
+  private final List<String> partitionPathFields;
+  private final List<String> recordKeyFields;
+  private final int minPayloadSize;
+
+  public UpdateConverter(String schemaStr, int minPayloadSize, List<String> partitionPathFields,
+      List<String> recordKeyFields) {
+    this.schemaStr = schemaStr;
+    this.partitionPathFields = partitionPathFields;
+    this.recordKeyFields = recordKeyFields;
+    this.minPayloadSize = minPayloadSize;
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> convert(JavaRDD<GenericRecord> inputRDD) {
+    return inputRDD.mapPartitions(recordItr -> new LazyRecordGeneratorIterator(new UpdateGeneratorIterator(recordItr,
+        schemaStr, partitionPathFields, recordKeyFields, minPayloadSize)));
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java
new file mode 100644
index 0000000..2889867
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java
@@ -0,0 +1,217 @@
+/*
+ * 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.hudi.integ.testsuite.dag;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator.Feature;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+
+/**
+ * Utility class to SerDe workflow dag.
+ */
+public class DagUtils {
+
+  static final ObjectMapper MAPPER = new ObjectMapper();
+
+  /**
+   * Converts a YAML path to {@link WorkflowDag}.
+   */
+  public static WorkflowDag convertYamlPathToDag(FileSystem fs, String path) throws IOException {
+    InputStream is = fs.open(new Path(path));
+    return convertYamlToDag(toString(is));
+  }
+
+  /**
+   * Converts a YAML representation to {@link WorkflowDag}.
+   */
+  public static WorkflowDag convertYamlToDag(String yaml) throws IOException {
+    Map<String, DagNode> allNodes = new HashMap<>();
+    final ObjectMapper yamlReader = new ObjectMapper(new YAMLFactory());
+    final JsonNode jsonNode = yamlReader.readTree(yaml);
+    Iterator<Entry<String, JsonNode>> itr = jsonNode.fields();
+    while (itr.hasNext()) {
+      Entry<String, JsonNode> dagNode = itr.next();
+      allNodes.put(dagNode.getKey(), convertJsonToDagNode(allNodes, dagNode.getValue()));
+    }
+    return new WorkflowDag(findRootNodes(allNodes));
+  }
+
+  /**
+   * Converts {@link WorkflowDag} to a YAML representation.
+   */
+  public static String convertDagToYaml(WorkflowDag dag) throws IOException {
+    final ObjectMapper yamlWriter = new ObjectMapper(new YAMLFactory().disable(Feature.WRITE_DOC_START_MARKER)
+        .enable(Feature.MINIMIZE_QUOTES).enable(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES));
+    JsonNode yamlNode = MAPPER.createObjectNode();
+    convertDagToYaml(yamlNode, dag.getNodeList());
+    return yamlWriter.writerWithDefaultPrettyPrinter().writeValueAsString(yamlNode);
+  }
+
+  private static void convertDagToYaml(JsonNode yamlNode, List<DagNode> dagNodes) throws IOException {
+    for (DagNode dagNode : dagNodes) {
+      String name = dagNode.getConfig().getOtherConfigs().getOrDefault(DeltaConfig.Config.NODE_NAME, dagNode.getName()).toString();
+      ((ObjectNode) yamlNode).put(name, convertDagNodeToJsonNode(dagNode));
+      if (dagNode.getChildNodes().size() > 0) {
+        convertDagToYaml(yamlNode, dagNode.getChildNodes());
+      }
+    }
+  }
+
+  private static DagNode convertJsonToDagNode(Map<String, DagNode> allNodes, JsonNode node) throws IOException {
+    String type = node.get(DeltaConfig.Config.TYPE).asText();
+    final DagNode retNode = convertJsonToDagNode(node, type);
+    Arrays.asList(node.get(DeltaConfig.Config.DEPENDENCIES).textValue().split(",")).stream().forEach(dep -> {
+      DagNode parentNode = allNodes.get(dep);
+      if (parentNode != null) {
+        parentNode.addChildNode(retNode);
+      }
+    });
+    return retNode;
+  }
+
+  private static List<DagNode> findRootNodes(Map<String, DagNode> allNodes) {
+    final List<DagNode> rootNodes = new ArrayList<>();
+    allNodes.entrySet().stream().forEach(entry -> {
+      if (entry.getValue().getParentNodes().size() < 1) {
+        rootNodes.add(entry.getValue());
+      }
+    });
+    return rootNodes;
+  }
+
+  private static DagNode convertJsonToDagNode(JsonNode node, String type) {
+    try {
+      DeltaConfig.Config config = DeltaConfig.Config.newBuilder().withConfigsMap(convertJsonNodeToMap(node)).build();
+      return (DagNode) ReflectionUtils.loadClass(generateFQN(type), config);
+    } catch (ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static String generateFQN(String name) throws ClassNotFoundException {
+    return Class.forName(StringUtils.joinUsingDelim(".",
+        DagNode.class.getName().substring(0, DagNode.class.getName().lastIndexOf(".")), name)).getName();
+  }
+
+  private static JsonNode convertDagNodeToJsonNode(DagNode node) throws IOException {
+    return createJsonNode(node, node.getClass().getSimpleName());
+  }
+
+  private static Map<String, Object> convertJsonNodeToMap(JsonNode node) {
+    Map<String, Object> configsMap = new HashMap<>();
+    Iterator<Entry<String, JsonNode>> itr = node.get(DeltaConfig.Config.CONFIG_NAME).fields();
+    while (itr.hasNext()) {
+      Entry<String, JsonNode> entry = itr.next();
+      switch (entry.getKey()) {
+        case DeltaConfig.Config.HIVE_QUERIES:
+          configsMap.put(DeltaConfig.Config.HIVE_QUERIES, getHiveQueries(entry));
+          break;
+        case DeltaConfig.Config.HIVE_PROPERTIES:
+          configsMap.put(DeltaConfig.Config.HIVE_PROPERTIES, getProperties(entry));
+          break;
+        default:
+          configsMap.put(entry.getKey(), getValue(entry.getValue()));
+          break;
+      }
+    }
+    return configsMap;
+  }
+
+  private static List<Pair<String, Integer>> getHiveQueries(Entry<String, JsonNode> entry) {
+    List<Pair<String, Integer>> queries = new ArrayList<>();
+    Iterator<Entry<String, JsonNode>> queriesItr = entry.getValue().fields();
+    while (queriesItr.hasNext()) {
+      queries.add(Pair.of(queriesItr.next().getValue().textValue(), queriesItr.next().getValue().asInt()));
+    }
+    return queries;
+  }
+
+  private static List<String> getProperties(Entry<String, JsonNode> entry) {
+    List<String> properties = new ArrayList<>();
+    Iterator<Entry<String, JsonNode>> queriesItr = entry.getValue().fields();
+    while (queriesItr.hasNext()) {
+      properties.add(queriesItr.next().getValue().textValue());
+    }
+    return properties;
+  }
+
+  private static Object getValue(JsonNode node) {
+    if (node.isInt()) {
+      return node.asInt();
+    } else if (node.isLong()) {
+      return node.asLong();
+    } else if (node.isShort()) {
+      return node.asInt();
+    } else if (node.isBoolean()) {
+      return node.asBoolean();
+    } else if (node.isDouble()) {
+      return node.asDouble();
+    } else if (node.isFloat()) {
+      return node.asDouble();
+    }
+    return node.textValue();
+  }
+
+  private static JsonNode createJsonNode(DagNode node, String type) throws IOException {
+    JsonNode configNode = MAPPER.readTree(node.getConfig().toString());
+    JsonNode jsonNode = MAPPER.createObjectNode();
+    ((ObjectNode) jsonNode).put(DeltaConfig.Config.CONFIG_NAME, configNode);
+    ((ObjectNode) jsonNode).put(DeltaConfig.Config.TYPE, type);
+    ((ObjectNode) jsonNode).put(DeltaConfig.Config.DEPENDENCIES, getDependencyNames(node));
+    return jsonNode;
+  }
+
+  private static String getDependencyNames(DagNode node) {
+    return node.getParentNodes().stream()
+        .map(e -> ((DagNode) e).getConfig().getOtherConfigs().getOrDefault(DeltaConfig.Config.NODE_NAME, node.getName()).toString())
+        .collect(Collectors.joining(",")).toString();
+  }
+
+  public static String toString(InputStream inputStream) throws IOException {
+    ByteArrayOutputStream result = new ByteArrayOutputStream();
+    byte[] buffer = new byte[1024];
+    int length;
+    while ((length = inputStream.read(buffer)) != -1) {
+      result.write(buffer, 0, length);
+    }
+    return result.toString("utf-8");
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/ExecutionContext.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/ExecutionContext.java
new file mode 100644
index 0000000..eecd763
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/ExecutionContext.java
@@ -0,0 +1,54 @@
+/*
+ * 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.hudi.integ.testsuite.dag;
+
+import java.io.Serializable;
+import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.generator.DeltaGenerator;
+import org.apache.spark.api.java.JavaSparkContext;
+
+/**
+ * This wraps the context needed for an execution of
+ * a {@link DagNode#execute(ExecutionContext)}.
+ */
+public class ExecutionContext implements Serializable {
+
+  private HoodieTestSuiteWriter hoodieTestSuiteWriter;
+  private DeltaGenerator deltaGenerator;
+  private transient JavaSparkContext jsc;
+
+  public ExecutionContext(JavaSparkContext jsc, HoodieTestSuiteWriter hoodieTestSuiteWriter, DeltaGenerator deltaGenerator) {
+    this.hoodieTestSuiteWriter = hoodieTestSuiteWriter;
+    this.deltaGenerator = deltaGenerator;
+    this.jsc = jsc;
+  }
+
+  public HoodieTestSuiteWriter getHoodieTestSuiteWriter() {
+    return hoodieTestSuiteWriter;
+  }
+
+  public DeltaGenerator getDeltaGenerator() {
+    return deltaGenerator;
+  }
+
+  public JavaSparkContext getJsc() {
+    return jsc;
+  }
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/SimpleWorkflowDagGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/SimpleWorkflowDagGenerator.java
new file mode 100644
index 0000000..ad6e9cb
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/SimpleWorkflowDagGenerator.java
@@ -0,0 +1,75 @@
+/*
+ * 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.hudi.integ.testsuite.dag;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.HiveQueryNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode;
+
+/**
+ * An example of how to generate a workflow dag programmatically. This is also used as the default workflow dag if
+ * none is provided.
+ */
+public class SimpleWorkflowDagGenerator implements WorkflowDagGenerator {
+
+  @Override
+  public WorkflowDag build() {
+
+    DagNode root = new InsertNode(DeltaConfig.Config.newBuilder()
+        .withNumRecordsToInsert(100)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(2)
+        .withRecordSize(1000).build());
+
+    DagNode child1 = new InsertNode(DeltaConfig.Config.newBuilder()
+        .withNumRecordsToInsert(100)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(2)
+        .withRecordSize(1000).build());
+
+    root.addChildNode(child1);
+
+    DagNode child1OfChild1 = new UpsertNode(DeltaConfig.Config.newBuilder()
+        .withNumRecordsToUpdate(100)
+        .withNumUpsertPartitions(2)
+        .withNumTimesToRepeat(1)
+        .withRecordSize(1000).build());
+
+    // Tests running 2 nodes in parallel
+    child1.addChildNode(child1OfChild1);
+
+    List<Pair<String, Integer>> queryAndResult = new ArrayList<>();
+    queryAndResult.add(Pair.of("select " + "count(*) from testdb1.table1 group "
+        + "by rider having count(*) < 1", 0));
+    DagNode child2OfChild1 = new HiveQueryNode(DeltaConfig.Config.newBuilder()
+        .withHiveQueryAndResults(queryAndResult).withHiveLocal(true).build());
+    child1.addChildNode(child2OfChild1);
+
+    List<DagNode> rootNodes = new ArrayList<>();
+    rootNodes.add(root);
+
+    return new WorkflowDag(rootNodes);
+  }
+
+}
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDag.java
similarity index 57%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDag.java
index 7dde326..e9171fc 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDag.java
@@ -16,28 +16,24 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
+package org.apache.hudi.integ.testsuite.dag;
 
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
-
-import org.apache.avro.generic.GenericRecord;
-
-import java.io.Serializable;
+import java.util.List;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
 
 /**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
+ * Workflow dag that encapsulates all execute nodes.
  */
-public abstract class KeyGenerator implements Serializable {
+public class WorkflowDag<O> {
 
-  protected transient TypedProperties config;
+  private List<DagNode<O>> nodeList;
+
+  public WorkflowDag(List<DagNode<O>> nodeList) {
+    this.nodeList = nodeList;
+  }
 
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
+  public List<DagNode<O>> getNodeList() {
+    return nodeList;
   }
 
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDagGenerator.java
similarity index 57%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDagGenerator.java
index 7dde326..98b086e 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/WorkflowDagGenerator.java
@@ -16,28 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
-
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
-
-import org.apache.avro.generic.GenericRecord;
-
-import java.io.Serializable;
+package org.apache.hudi.integ.testsuite.dag;
 
 /**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
+ * A interface which represents a workflow dag generator.
  */
-public abstract class KeyGenerator implements Serializable {
-
-  protected transient TypedProperties config;
-
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
-  }
+public interface WorkflowDagGenerator {
 
   /**
-   * Generate a Hoodie Key out of provided generic record.
+   * Builds a {@link WorkflowDag}.
    */
-  public abstract HoodieKey getKey(GenericRecord record);
+  WorkflowDag build();
+
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BulkInsertNode.java
similarity index 54%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BulkInsertNode.java
index 7dde326..7a8f405 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BulkInsertNode.java
@@ -16,28 +16,25 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
+package org.apache.hudi.integ.testsuite.dag.nodes;
 
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.spark.api.java.JavaRDD;
 
-import org.apache.avro.generic.GenericRecord;
+public class BulkInsertNode extends InsertNode {
 
-import java.io.Serializable;
-
-/**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
- */
-public abstract class KeyGenerator implements Serializable {
-
-  protected transient TypedProperties config;
+  public BulkInsertNode(Config config) {
+    super(config);
+  }
 
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
+  @Override
+  protected JavaRDD<WriteStatus> ingest(HoodieTestSuiteWriter hoodieTestSuiteWriter, Option<String> commitTime)
+      throws Exception {
+    log.info("Execute bulk ingest node {}", this.getName());
+    return hoodieTestSuiteWriter.bulkInsert(commitTime);
   }
 
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CleanNode.java
similarity index 56%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CleanNode.java
index 7dde326..7083b47 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CleanNode.java
@@ -16,28 +16,19 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
+package org.apache.hudi.integ.testsuite.dag.nodes;
 
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
 
-import org.apache.avro.generic.GenericRecord;
+public class CleanNode extends DagNode<Boolean> {
 
-import java.io.Serializable;
-
-/**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
- */
-public abstract class KeyGenerator implements Serializable {
-
-  protected transient TypedProperties config;
+  public CleanNode() {
+  }
 
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
+  @Override
+  public void execute(ExecutionContext executionContext) throws Exception {
+    log.info("Executing clean node {}", this.getName());
+    executionContext.getHoodieTestSuiteWriter().getWriteClient(this).clean();
   }
 
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
 }
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java
new file mode 100644
index 0000000..92fe53c
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java
@@ -0,0 +1,47 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+import org.apache.spark.api.java.JavaRDD;
+
+public class CompactNode extends DagNode<JavaRDD<WriteStatus>> {
+
+  public CompactNode(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext) throws Exception {
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(executionContext.getHoodieTestSuiteWriter().getConfiguration(),
+        executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath);
+    Option<HoodieInstant> lastInstant = metaClient.getActiveTimeline()
+        .getCommitsAndCompactionTimeline().filterPendingCompactionTimeline().lastInstant();
+    if (lastInstant.isPresent()) {
+      log.info("Compacting instant {}", lastInstant.get());
+      this.result = executionContext.getHoodieTestSuiteWriter().compact(Option.of(lastInstant.get().getTimestamp()));
+    }
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DagNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DagNode.java
new file mode 100644
index 0000000..aa54dc9
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/DagNode.java
@@ -0,0 +1,125 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Represents a Node in the DAG of operations for a workflow.
+ */
+public abstract class DagNode<O> implements Comparable<DagNode<O>> {
+
+  protected static Logger log = LoggerFactory.getLogger(DagNode.class);
+
+  protected List<DagNode<O>> childNodes;
+  protected List<DagNode<O>> parentNodes;
+  protected O result;
+  protected Config config;
+  private boolean isCompleted;
+
+  public DagNode<O> addChildNode(DagNode childNode) {
+    childNode.getParentNodes().add(this);
+    getChildNodes().add(childNode);
+    return this;
+  }
+
+  public DagNode<O> addParentNode(DagNode parentNode) {
+    if (!this.getParentNodes().contains(parentNode)) {
+      this.getParentNodes().add(parentNode);
+    }
+    return this;
+  }
+
+  public O getResult() {
+    return result;
+  }
+
+  public List<DagNode<O>> getChildNodes() {
+    if (childNodes == null) {
+      childNodes = new LinkedList<>();
+    }
+    return childNodes;
+  }
+
+  public List<DagNode<O>> getParentNodes() {
+    if (parentNodes == null) {
+      this.parentNodes = new ArrayList<>();
+    }
+    return this.parentNodes;
+  }
+
+  public void setParentNodes(List<DagNode<O>> parentNodes) {
+    this.parentNodes = parentNodes;
+  }
+
+  public abstract void execute(ExecutionContext context) throws Exception;
+
+  public boolean isCompleted() {
+    return isCompleted;
+  }
+
+  public void setCompleted(boolean completed) {
+    isCompleted = completed;
+  }
+
+  public Config getConfig() {
+    return config;
+  }
+
+  public String getName() {
+    Object name = this.config.getOtherConfigs().get(Config.NODE_NAME);
+    if (name == null) {
+      String randomName = UUID.randomUUID().toString();
+      this.config.getOtherConfigs().put(Config.NODE_NAME, randomName);
+      return randomName;
+    }
+    return name.toString();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    DagNode<?> dagNode = (DagNode<?>) o;
+    return getName() == dagNode.getName();
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(getName());
+  }
+
+  @Override
+  public int compareTo(DagNode<O> thatNode) {
+    return this.hashCode() - thatNode.hashCode();
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java
new file mode 100644
index 0000000..04f8c2e
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java
@@ -0,0 +1,87 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import org.apache.hudi.DataSourceUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.hive.HiveSyncConfig;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider;
+
+public class HiveQueryNode extends DagNode<Boolean> {
+
+  private HiveServiceProvider hiveServiceProvider;
+
+  public HiveQueryNode(DeltaConfig.Config config) {
+    this.config = config;
+    this.hiveServiceProvider = new HiveServiceProvider(config);
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext) throws Exception {
+    log.info("Executing hive query node {}", this.getName());
+    this.hiveServiceProvider.startLocalHiveServiceIfNeeded(executionContext.getHoodieTestSuiteWriter().getConfiguration());
+    HiveSyncConfig hiveSyncConfig = DataSourceUtils
+        .buildHiveSyncConfig(executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
+                .getDeltaSyncService().getDeltaSync().getProps(),
+            executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
+                .getDeltaSyncService().getDeltaSync().getCfg().targetBasePath,
+            executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
+                .getDeltaSyncService().getDeltaSync().getCfg().baseFileFormat);
+    this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter());
+    Connection con = DriverManager.getConnection(hiveSyncConfig.jdbcUrl, hiveSyncConfig.hiveUser,
+        hiveSyncConfig.hivePass);
+    Statement stmt = con.createStatement();
+    stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat");
+    for (String hiveProperty : this.config.getHiveProperties()) {
+      executeStatement(hiveProperty, stmt);
+    }
+    for (Pair<String, Integer> queryAndResult : this.config.getHiveQueries()) {
+      log.info("Running {}", queryAndResult.getLeft());
+      ResultSet res = stmt.executeQuery(queryAndResult.getLeft());
+      if (!res.next()) {
+        log.info("res.next() was False - typically this means the query returned no rows.");
+        assert 0 == queryAndResult.getRight();
+      } else {
+        Integer result = res.getInt(1);
+        if (!queryAndResult.getRight().equals(result)) {
+          throw new AssertionError(
+              "QUERY: " + queryAndResult.getLeft()
+                  + " | EXPECTED RESULT = " + queryAndResult.getRight()
+                  + " | ACTUAL RESULT = " + result
+          );
+        }
+      }
+      log.info("Successfully validated query!");
+    }
+    this.hiveServiceProvider.stopLocalHiveServiceIfNeeded();
+  }
+
+  private void executeStatement(String query, Statement stmt) throws SQLException {
+    log.info("Executing statement {}", stmt.toString());
+    stmt.execute(query);
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveSyncNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveSyncNode.java
new file mode 100644
index 0000000..f24da14
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveSyncNode.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+public class HiveSyncNode extends DagNode<Boolean> {
+
+  private HiveServiceProvider hiveServiceProvider;
+
+  public HiveSyncNode(Config config) {
+    this.config = config;
+    this.hiveServiceProvider = new HiveServiceProvider(config);
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext) throws Exception {
+    log.info("Executing hive sync node");
+    this.hiveServiceProvider.startLocalHiveServiceIfNeeded(executionContext.getHoodieTestSuiteWriter().getConfiguration());
+    this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter());
+    executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync().syncHive();
+    this.hiveServiceProvider.stopLocalHiveServiceIfNeeded();
+  }
+
+  public HiveServiceProvider getHiveServiceProvider() {
+    return hiveServiceProvider;
+  }
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/InsertNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/InsertNode.java
new file mode 100644
index 0000000..23cb285
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/InsertNode.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+import org.apache.hudi.integ.testsuite.generator.DeltaGenerator;
+import org.apache.spark.api.java.JavaRDD;
+
+public class InsertNode extends DagNode<JavaRDD<WriteStatus>> {
+
+  public InsertNode(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext) throws Exception {
+    generate(executionContext.getDeltaGenerator());
+    log.info("Configs : {}", this.config);
+    if (!config.isDisableIngest()) {
+      log.info("Inserting input data {}", this.getName());
+      Option<String> commitTime = executionContext.getHoodieTestSuiteWriter().startCommit();
+      JavaRDD<WriteStatus> writeStatus = ingest(executionContext.getHoodieTestSuiteWriter(), commitTime);
+      executionContext.getHoodieTestSuiteWriter().commit(writeStatus, commitTime);
+      this.result = writeStatus;
+    }
+  }
+
+  protected void generate(DeltaGenerator deltaGenerator) throws Exception {
+    if (!config.isDisableGenerate()) {
+      log.info("Generating input data for node {}", this.getName());
+      deltaGenerator.writeRecords(deltaGenerator.generateInserts(config)).count();
+    }
+  }
+
+  protected JavaRDD<WriteStatus> ingest(HoodieTestSuiteWriter hoodieTestSuiteWriter,
+      Option<String> commitTime) throws Exception {
+    return hoodieTestSuiteWriter.insert(commitTime);
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/RollbackNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/RollbackNode.java
new file mode 100644
index 0000000..b6d828a
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/RollbackNode.java
@@ -0,0 +1,48 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+public class RollbackNode extends DagNode<Option<HoodieInstant>> {
+
+  public RollbackNode(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext) throws Exception {
+    log.info("Executing rollback node {}", this.getName());
+    // Can only be done with an instantiation of a new WriteClient hence cannot be done during DeltaStreamer
+    // testing for now
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(executionContext.getHoodieTestSuiteWriter().getConfiguration(),
+        executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath);
+    Option<HoodieInstant> lastInstant = metaClient.getActiveTimeline().getCommitsTimeline().lastInstant();
+    if (lastInstant.isPresent()) {
+      log.info("Rolling back last instant {}", lastInstant.get());
+      executionContext.getHoodieTestSuiteWriter().getWriteClient(this).rollback(lastInstant.get().getTimestamp());
+      this.result = lastInstant;
+    }
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ScheduleCompactNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ScheduleCompactNode.java
new file mode 100644
index 0000000..93502ae
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ScheduleCompactNode.java
@@ -0,0 +1,56 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+public class ScheduleCompactNode extends DagNode<Option<String>> {
+
+  public ScheduleCompactNode(Config config) {
+    this.config = config;
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext) throws Exception {
+    log.info("Executing schedule compact node {}", this.getName());
+    // Can only be done with an instantiation of a new WriteClient hence cannot be done during DeltaStreamer
+    // testing for now
+    // Find the last commit and extra the extra metadata to be passed to the schedule compaction. This is
+    // done to ensure the CHECKPOINT is correctly passed from commit to commit
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(executionContext.getHoodieTestSuiteWriter().getConfiguration(),
+        executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath);
+    Option<HoodieInstant> lastInstant = metaClient.getActiveTimeline().getCommitsTimeline().lastInstant();
+    if (lastInstant.isPresent()) {
+      HoodieCommitMetadata metadata = org.apache.hudi.common.model.HoodieCommitMetadata.fromBytes(metaClient
+          .getActiveTimeline().getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class);
+      Option<String> scheduledInstant = executionContext.getHoodieTestSuiteWriter().scheduleCompaction(Option.of(metadata
+              .getExtraMetadata()));
+      if (scheduledInstant.isPresent()) {
+        log.info("Scheduling compaction instant {}", scheduledInstant.get());
+      }
+      this.result = scheduledInstant;
+    }
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/SparkSQLQueryNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/SparkSQLQueryNode.java
new file mode 100644
index 0000000..a8a1b72
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/SparkSQLQueryNode.java
@@ -0,0 +1,61 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+public class SparkSQLQueryNode extends DagNode<Boolean> {
+
+  HiveServiceProvider hiveServiceProvider;
+
+  public SparkSQLQueryNode(Config config) {
+    this.config = config;
+    this.hiveServiceProvider = new HiveServiceProvider(config);
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext) throws Exception {
+    log.info("Executing spark sql query node");
+    this.hiveServiceProvider.startLocalHiveServiceIfNeeded(executionContext.getHoodieTestSuiteWriter().getConfiguration());
+    this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter());
+    SparkSession session = SparkSession.builder().sparkContext(executionContext.getJsc().sc()).getOrCreate();
+    for (String hiveProperty : this.config.getHiveProperties()) {
+      session.sql(hiveProperty).count();
+    }
+    for (Pair<String, Integer> queryAndResult : this.config.getHiveQueries()) {
+      log.info("Running {}", queryAndResult.getLeft());
+      Dataset<Row> res = session.sql(queryAndResult.getLeft());
+      if (res.count() == 0) {
+        assert 0 == queryAndResult.getRight();
+      } else {
+        assert ((Row[]) res.collect())[0].getInt(0) == queryAndResult.getRight();
+      }
+      log.info("Successfully validated query!");
+    }
+    this.hiveServiceProvider.stopLocalHiveServiceIfNeeded();
+    this.result = true;
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/UpsertNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/UpsertNode.java
new file mode 100644
index 0000000..3872f76
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/UpsertNode.java
@@ -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.
+ */
+
+package org.apache.hudi.integ.testsuite.dag.nodes;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.generator.DeltaGenerator;
+import org.apache.spark.api.java.JavaRDD;
+
+public class UpsertNode extends InsertNode {
+
+  public UpsertNode(Config config) {
+    super(config);
+  }
+
+  @Override
+  protected void generate(DeltaGenerator deltaGenerator) throws Exception {
+    if (!config.isDisableGenerate()) {
+      log.info("Generating input data {}", this.getName());
+      deltaGenerator.writeRecords(deltaGenerator.generateUpdates(config)).count();
+    }
+  }
+
+  @Override
+  protected JavaRDD<WriteStatus> ingest(HoodieTestSuiteWriter hoodieTestSuiteWriter, Option<String> commitTime)
+      throws Exception {
+    if (!config.isDisableIngest()) {
+      log.info("Upserting input data {}", this.getName());
+      this.result = hoodieTestSuiteWriter.upsert(commitTime);
+    }
+    return this.result;
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateNode.java
new file mode 100644
index 0000000..5ba0249
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateNode.java
@@ -0,0 +1,48 @@
+/*
+ * 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.hudi.integ.testsuite.dag.nodes;
+
+import java.util.List;
+import java.util.function.Function;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+
+public class ValidateNode<R> extends DagNode {
+
+  protected Function<List<DagNode>, R> function;
+
+  public ValidateNode(Config config, Function<List<DagNode>, R> function) {
+    this.function = function;
+    this.config = config;
+  }
+
+  @Override
+  public void execute(ExecutionContext executionContext) {
+    if (this.getParentNodes().size() > 0 && (Boolean) this.config.getOtherConfigs().getOrDefault("WAIT_FOR_PARENTS",
+        true)) {
+      for (DagNode node : (List<DagNode>) this.getParentNodes()) {
+        if (!node.isCompleted()) {
+          throw new RuntimeException("cannot validate before parent nodes are complete");
+        }
+      }
+    }
+    this.result = this.function.apply((List<DagNode>) this.getParentNodes());
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java
new file mode 100644
index 0000000..7d78d83
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.integ.testsuite.dag.scheduler;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter;
+import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
+import org.apache.hudi.integ.testsuite.dag.WorkflowDag;
+import org.apache.hudi.integ.testsuite.generator.DeltaGenerator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DagScheduler {
+
+  private static Logger log = LoggerFactory.getLogger(DagScheduler.class);
+  private WorkflowDag workflowDag;
+  private ExecutionContext executionContext;
+
+  public DagScheduler(WorkflowDag workflowDag, HoodieTestSuiteWriter hoodieTestSuiteWriter, DeltaGenerator deltaGenerator) {
+    this.workflowDag = workflowDag;
+    this.executionContext = new ExecutionContext(null, hoodieTestSuiteWriter, deltaGenerator);
+  }
+
+  public void schedule() throws Exception {
+    ExecutorService service = Executors.newFixedThreadPool(2);
+    try {
+      execute(service, workflowDag.getNodeList());
+      service.shutdown();
+    } finally {
+      if (!service.isShutdown()) {
+        log.info("Forcing shutdown of executor service, this might kill running tasks");
+        service.shutdownNow();
+      }
+    }
+  }
+
+  private void execute(ExecutorService service, List<DagNode> nodes) throws Exception {
+    // Nodes at the same level are executed in parallel
+    Queue<DagNode> queue = new PriorityQueue<>(nodes);
+    log.info("Running workloads");
+    do {
+      List<Future> futures = new ArrayList<>();
+      Set<DagNode> childNodes = new HashSet<>();
+      while (queue.size() > 0) {
+        DagNode nodeToExecute = queue.poll();
+        futures.add(service.submit(() -> executeNode(nodeToExecute)));
+        if (nodeToExecute.getChildNodes().size() > 0) {
+          childNodes.addAll(nodeToExecute.getChildNodes());
+        }
+      }
+      queue.addAll(childNodes);
+      childNodes.clear();
+      for (Future future : futures) {
+        future.get(1, TimeUnit.HOURS);
+      }
+    } while (queue.size() > 0);
+    log.info("Finished workloads");
+  }
+
+  private void executeNode(DagNode node) {
+    if (node.isCompleted()) {
+      throw new RuntimeException("DagNode already completed! Cannot re-execute");
+    }
+    try {
+      node.execute(executionContext);
+      node.setCompleted(true);
+      log.info("Finished executing {}", node.getName());
+    } catch (Exception e) {
+      log.error("Exception executing node");
+      throw new HoodieException(e);
+    }
+  }
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java
new file mode 100644
index 0000000..24311e0
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/DeltaGenerator.java
@@ -0,0 +1,237 @@
+/*
+ * 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.hudi.integ.testsuite.generator;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.StreamSupport;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.converter.Converter;
+import org.apache.hudi.integ.testsuite.converter.UpdateConverter;
+import org.apache.hudi.integ.testsuite.reader.DFSAvroDeltaInputReader;
+import org.apache.hudi.integ.testsuite.reader.DFSHoodieDatasetInputReader;
+import org.apache.hudi.integ.testsuite.reader.DeltaInputReader;
+import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
+import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats;
+import org.apache.hudi.integ.testsuite.writer.DeltaWriterAdapter;
+import org.apache.hudi.integ.testsuite.writer.DeltaWriterFactory;
+import org.apache.hudi.keygen.ComplexKeyGenerator;
+import org.apache.hudi.keygen.KeyGenerator;
+import org.apache.hudi.keygen.SimpleKeyGenerator;
+import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.storage.StorageLevel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Tuple2;
+
+/**
+ * The delta generator generates all types of workloads (insert, update) for the given configs.
+ */
+public class DeltaGenerator implements Serializable {
+
+  private static Logger log = LoggerFactory.getLogger(DFSHoodieDatasetInputReader.class);
+
+  private DeltaConfig deltaOutputConfig;
+  private transient JavaSparkContext jsc;
+  private transient SparkSession sparkSession;
+  private String schemaStr;
+  private List<String> recordRowKeyFieldNames;
+  private List<String> partitionPathFieldNames;
+  private int batchId;
+
+  public DeltaGenerator(DeltaConfig deltaOutputConfig, JavaSparkContext jsc, SparkSession sparkSession,
+      String schemaStr,
+      KeyGenerator keyGenerator) {
+    this.deltaOutputConfig = deltaOutputConfig;
+    this.jsc = jsc;
+    this.sparkSession = sparkSession;
+    this.schemaStr = schemaStr;
+    this.recordRowKeyFieldNames = keyGenerator instanceof ComplexKeyGenerator ? ((ComplexKeyGenerator) keyGenerator)
+        .getRecordKeyFields() : Arrays.asList(((SimpleKeyGenerator) keyGenerator).getRecordKeyField());
+    this.partitionPathFieldNames = keyGenerator instanceof ComplexKeyGenerator ? ((ComplexKeyGenerator) keyGenerator)
+        .getPartitionPathFields() : Arrays.asList(((SimpleKeyGenerator) keyGenerator).getPartitionPathField());
+  }
+
+  public JavaRDD<DeltaWriteStats> writeRecords(JavaRDD<GenericRecord> records) {
+    // The following creates a new anonymous function for iterator and hence results in serialization issues
+    JavaRDD<DeltaWriteStats> ws = records.mapPartitions(itr -> {
+      try {
+        DeltaWriterAdapter<GenericRecord> deltaWriterAdapter = DeltaWriterFactory
+            .getDeltaWriterAdapter(deltaOutputConfig, batchId);
+        return Collections.singletonList(deltaWriterAdapter.write(itr)).iterator();
+      } catch (IOException io) {
+        throw new UncheckedIOException(io);
+      }
+    }).flatMap(List::iterator);
+    batchId++;
+    return ws;
+  }
+
+  public JavaRDD<GenericRecord> generateInserts(Config operation) {
+    long recordsPerPartition = operation.getNumRecordsInsert();
+    int minPayloadSize = operation.getRecordSize();
+    JavaRDD<GenericRecord> inputBatch = jsc.parallelize(Collections.EMPTY_LIST)
+        .repartition(operation.getNumInsertPartitions()).mapPartitions(p -> {
+          return new LazyRecordGeneratorIterator(new FlexibleSchemaRecordGenerationIterator(recordsPerPartition,
+              minPayloadSize, schemaStr, partitionPathFieldNames));
+        });
+    return inputBatch;
+  }
+
+  public JavaRDD<GenericRecord> generateUpdates(Config config) throws IOException {
+    if (deltaOutputConfig.getDeltaOutputMode() == DeltaOutputMode.DFS) {
+      JavaRDD<GenericRecord> inserts = null;
+      if (config.getNumRecordsInsert() > 0) {
+        inserts = generateInserts(config);
+      }
+      DeltaInputReader deltaInputReader = null;
+      JavaRDD<GenericRecord> adjustedRDD = null;
+      if (config.getNumUpsertPartitions() < 1) {
+        // randomly generate updates for a given number of records without regard to partitions and files
+        deltaInputReader = new DFSAvroDeltaInputReader(sparkSession, schemaStr,
+            ((DFSDeltaConfig) deltaOutputConfig).getDeltaBasePath(), Option.empty(), Option.empty());
+        adjustedRDD = deltaInputReader.read(config.getNumRecordsUpsert());
+        adjustedRDD = adjustRDDToGenerateExactNumUpdates(adjustedRDD, jsc, config.getNumRecordsUpsert());
+      } else {
+        deltaInputReader =
+            new DFSHoodieDatasetInputReader(jsc, ((DFSDeltaConfig) deltaOutputConfig).getDatasetOutputPath(),
+                schemaStr);
+        if (config.getFractionUpsertPerFile() > 0) {
+          adjustedRDD = deltaInputReader.read(config.getNumUpsertPartitions(), config.getNumUpsertFiles(),
+              config.getFractionUpsertPerFile());
+        } else {
+          adjustedRDD = deltaInputReader.read(config.getNumUpsertPartitions(), config.getNumUpsertFiles(), config
+              .getNumRecordsUpsert());
+        }
+      }
+      log.info("Repartitioning records");
+      // persist this since we will make multiple passes over this
+      adjustedRDD = adjustedRDD.repartition(jsc.defaultParallelism());
+      log.info("Repartitioning records done");
+      Converter converter = new UpdateConverter(schemaStr, config.getRecordSize(),
+          partitionPathFieldNames, recordRowKeyFieldNames);
+      JavaRDD<GenericRecord> updates = converter.convert(adjustedRDD);
+      log.info("Records converted");
+      updates.persist(StorageLevel.DISK_ONLY());
+      return inserts != null ? inserts.union(updates) : updates;
+      // TODO : Generate updates for only N partitions.
+    } else {
+      throw new IllegalArgumentException("Other formats are not supported at the moment");
+    }
+  }
+
+  public Map<Integer, Long> getPartitionToCountMap(JavaRDD<GenericRecord> records) {
+    // Requires us to keep the partitioner the same
+    return records.mapPartitionsWithIndex((index, itr) -> {
+      Iterable<GenericRecord> newIterable = () -> itr;
+      // parallelize counting for speed
+      long count = StreamSupport.stream(newIterable.spliterator(), true).count();
+      return Arrays.asList(new Tuple2<>(index, count)).iterator();
+    }, true).mapToPair(i -> i).collectAsMap();
+  }
+
+  public Map<Integer, Long> getAdjustedPartitionsCount(Map<Integer, Long> partitionCountMap, long
+      recordsToRemove) {
+    long remainingRecordsToRemove = recordsToRemove;
+    Iterator<Map.Entry<Integer, Long>> iterator = partitionCountMap.entrySet().iterator();
+    Map<Integer, Long> adjustedPartitionCountMap = new HashMap<>();
+    while (iterator.hasNext()) {
+      Map.Entry<Integer, Long> entry = iterator.next();
+      if (entry.getValue() < remainingRecordsToRemove) {
+        remainingRecordsToRemove -= entry.getValue();
+        adjustedPartitionCountMap.put(entry.getKey(), 0L);
+      } else {
+        long newValue = entry.getValue() - remainingRecordsToRemove;
+        remainingRecordsToRemove = 0;
+        adjustedPartitionCountMap.put(entry.getKey(), newValue);
+      }
+      if (remainingRecordsToRemove == 0) {
+        break;
+      }
+    }
+    return adjustedPartitionCountMap;
+  }
+
+  public JavaRDD<GenericRecord> adjustRDDToGenerateExactNumUpdates(JavaRDD<GenericRecord> updates, JavaSparkContext
+      jsc, long totalRecordsRequired) {
+    Map<Integer, Long> actualPartitionCountMap = getPartitionToCountMap(updates);
+    long totalRecordsGenerated = actualPartitionCountMap.values().stream().mapToLong(Long::longValue).sum();
+    if (isSafeToTake(totalRecordsRequired, totalRecordsGenerated)) {
+      // Generate totalRecordsRequired - totalRecordsGenerated new records and union the RDD's
+      // NOTE : This performs poorly when totalRecordsRequired >> totalRecordsGenerated. Hence, always
+      // ensure that enough inserts are created before hand (this needs to be noted during the WorkflowDag creation)
+      long sizeOfUpdateRDD = totalRecordsGenerated;
+      while (totalRecordsRequired != sizeOfUpdateRDD) {
+        long recordsToTake = (totalRecordsRequired - sizeOfUpdateRDD) > sizeOfUpdateRDD
+            ? sizeOfUpdateRDD : (totalRecordsRequired - sizeOfUpdateRDD);
+        if ((totalRecordsRequired - sizeOfUpdateRDD) > recordsToTake && recordsToTake <= sizeOfUpdateRDD) {
+          updates = updates.union(updates);
+          sizeOfUpdateRDD *= 2;
+        } else {
+          List<GenericRecord> remainingUpdates = updates.take((int) (recordsToTake));
+          updates = updates.union(jsc.parallelize(remainingUpdates));
+          sizeOfUpdateRDD = sizeOfUpdateRDD + recordsToTake;
+        }
+      }
+      return updates;
+    } else if (totalRecordsRequired < totalRecordsGenerated) {
+      final Map<Integer, Long> adjustedPartitionCountMap = getAdjustedPartitionsCount(actualPartitionCountMap,
+          totalRecordsGenerated - totalRecordsRequired);
+      // limit counts across partitions to meet the exact number of updates required
+      JavaRDD<GenericRecord> trimmedRecords = updates.mapPartitionsWithIndex((index, itr) -> {
+        int counter = 1;
+        List<GenericRecord> entriesToKeep = new ArrayList<>();
+        if (!adjustedPartitionCountMap.containsKey(index)) {
+          return itr;
+        } else {
+          long recordsToKeepForThisPartition = adjustedPartitionCountMap.get(index);
+          while (counter <= recordsToKeepForThisPartition && itr.hasNext()) {
+            entriesToKeep.add(itr.next());
+            counter++;
+          }
+          return entriesToKeep.iterator();
+        }
+      }, true);
+      return trimmedRecords;
+    }
+    return updates;
+  }
+
+  private boolean isSafeToTake(long totalRecords, long totalRecordsGenerated) {
+    // TODO : Ensure that the difference between totalRecords and totalRecordsGenerated is not too big, if yes,
+    // then there are fewer number of records on disk, hence we need to find another way to generate updates when
+    // requiredUpdates >> insertedRecords
+    return totalRecords > totalRecordsGenerated;
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java
new file mode 100644
index 0000000..614add5
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/FlexibleSchemaRecordGenerationIterator.java
@@ -0,0 +1,70 @@
+/*
+ * 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.hudi.integ.testsuite.generator;
+
+import java.util.Iterator;
+import java.util.List;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+
+/**
+ * A GenericRecordGeneratorIterator for the custom schema of the workload. Implements {@link Iterator} to allow for
+ * iteration semantics.
+ */
+public class FlexibleSchemaRecordGenerationIterator implements Iterator<GenericRecord> {
+
+  // Stores how many records to generate as part of this iterator. Ideally, one iterator is started per spark
+  // partition.
+  private long counter;
+  // Use the full payload generator as default
+  private GenericRecordFullPayloadGenerator generator;
+  // Store last record for the partition path of the first payload to be used for all subsequent generated payloads
+  private GenericRecord lastRecord;
+  // Partition path field name
+  private List<String> partitionPathFieldNames;
+
+  public FlexibleSchemaRecordGenerationIterator(long maxEntriesToProduce, String schema) {
+    this(maxEntriesToProduce, GenericRecordFullPayloadGenerator.DEFAULT_PAYLOAD_SIZE, schema, null);
+  }
+
+  public FlexibleSchemaRecordGenerationIterator(long maxEntriesToProduce, int minPayloadSize, String schemaStr,
+      List<String> partitionPathFieldNames) {
+    this.counter = maxEntriesToProduce;
+    this.partitionPathFieldNames = partitionPathFieldNames;
+    Schema schema = new Schema.Parser().parse(schemaStr);
+    this.generator = new GenericRecordFullPayloadGenerator(schema, minPayloadSize);
+  }
+
+  @Override
+  public boolean hasNext() {
+    return this.counter > 0;
+  }
+
+  @Override
+  public GenericRecord next() {
+    this.counter--;
+    if (lastRecord == null) {
+      GenericRecord record = this.generator.getNewPayload();
+      lastRecord = record;
+      return record;
+    } else {
+      return this.generator.randomize(lastRecord, this.partitionPathFieldNames);
+    }
+  }
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java
new file mode 100644
index 0000000..df1a4c7
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadGenerator.java
@@ -0,0 +1,285 @@
+/*
+ * 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.hudi.integ.testsuite.generator;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.avro.LogicalType;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Type;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericData.Fixed;
+import org.apache.avro.generic.GenericFixed;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.common.util.collection.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This is a GenericRecord payload generator that generates full generic records {@link GenericRecord}.
+ * Every field of a generic record created using this generator contains a random value.
+ */
+public class GenericRecordFullPayloadGenerator implements Serializable {
+
+  public static final int DEFAULT_PAYLOAD_SIZE = 1024 * 10; // 10 KB
+  private static Logger log = LoggerFactory.getLogger(GenericRecordFullPayloadGenerator.class);
+  protected final Random random = new Random();
+  // The source schema used to generate a payload
+  private final transient Schema baseSchema;
+  // Used to validate a generic record
+  private final transient GenericData genericData = new GenericData();
+  // Number of more bytes to add based on the estimated full record payload size and min payload size
+  private int numberOfBytesToAdd;
+  // If more elements should be packed to meet the minPayloadSize
+  private boolean shouldAddMore;
+  // How many complex fields have we visited that can help us pack more entries and increase the size of the record
+  private int numberOfComplexFields;
+  // The size of a full record where every field of a generic record created contains 1 random value
+  private int estimatedFullPayloadSize;
+  // LogicalTypes in Avro 1.8.2
+  private static final String DECIMAL = "decimal";
+  private static final String UUID_NAME = "uuid";
+  private static final String DATE = "date";
+  private static final String TIME_MILLIS = "time-millis";
+  private static final String TIME_MICROS = "time-micros";
+  private static final String TIMESTAMP_MILLIS = "timestamp-millis";
+  private static final String TIMESTAMP_MICROS = "timestamp-micros";
+
+  public GenericRecordFullPayloadGenerator(Schema schema) {
+    this(schema, DEFAULT_PAYLOAD_SIZE);
+  }
+
+  public GenericRecordFullPayloadGenerator(Schema schema, int minPayloadSize) {
+    Pair<Integer, Integer> sizeInfo = new GenericRecordFullPayloadSizeEstimator(schema)
+        .typeEstimateAndNumComplexFields();
+    this.estimatedFullPayloadSize = sizeInfo.getLeft();
+    this.numberOfComplexFields = sizeInfo.getRight();
+    this.baseSchema = schema;
+    this.shouldAddMore = estimatedFullPayloadSize < minPayloadSize;
+    if (this.shouldAddMore) {
+      this.numberOfBytesToAdd = minPayloadSize - estimatedFullPayloadSize;
+      if (numberOfComplexFields < 1) {
+        log.warn("The schema does not have any collections/complex fields. Cannot achieve minPayloadSize : {}",
+            minPayloadSize);
+      }
+    }
+  }
+
+  protected static boolean isPrimitive(Schema localSchema) {
+    if (localSchema.getType() != Type.ARRAY
+        && localSchema.getType() != Type.MAP
+        && localSchema.getType() != Type.RECORD
+        && localSchema.getType() != Type.UNION) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  public GenericRecord getNewPayload() {
+    return convert(baseSchema);
+  }
+
+  public GenericRecord getUpdatePayload(GenericRecord record, List<String> blacklistFields) {
+    return randomize(record, blacklistFields);
+  }
+
+  protected GenericRecord convert(Schema schema) {
+    GenericRecord result = new GenericData.Record(schema);
+    for (Schema.Field f : schema.getFields()) {
+      result.put(f.name(), typeConvert(f.schema()));
+    }
+    return result;
+  }
+
+  protected GenericRecord convertPartial(Schema schema) {
+    GenericRecord result = new GenericData.Record(schema);
+    for (Schema.Field f : schema.getFields()) {
+      boolean setNull = random.nextBoolean();
+      if (!setNull) {
+        result.put(f.name(), typeConvert(f.schema()));
+      } else {
+        result.put(f.name(), null);
+      }
+    }
+    // TODO : pack remaining bytes into a complex field
+    return result;
+  }
+
+  protected GenericRecord randomize(GenericRecord record, List<String> blacklistFields) {
+    for (Schema.Field f : record.getSchema().getFields()) {
+      if (blacklistFields == null || !blacklistFields.contains(f.name())) {
+        record.put(f.name(), typeConvert(f.schema()));
+      }
+    }
+    return record;
+  }
+
+  private Object typeConvert(Schema schema) {
+    Schema localSchema = schema;
+    if (isOption(schema)) {
+      localSchema = getNonNull(schema);
+    }
+    switch (localSchema.getType()) {
+      case BOOLEAN:
+        return random.nextBoolean();
+      case DOUBLE:
+        return random.nextDouble();
+      case FLOAT:
+        return random.nextFloat();
+      case INT:
+        return random.nextInt();
+      case LONG:
+        return random.nextLong();
+      case STRING:
+        return UUID.randomUUID().toString();
+      case ENUM:
+        List<String> enumSymbols = localSchema.getEnumSymbols();
+        return new GenericData.EnumSymbol(localSchema, enumSymbols.get(random.nextInt(enumSymbols.size() - 1)));
+      case RECORD:
+        return convert(localSchema);
+      case ARRAY:
+        Schema elementSchema = localSchema.getElementType();
+        List listRes = new ArrayList();
+        if (isPrimitive(elementSchema) && this.shouldAddMore) {
+          int numEntriesToAdd = numEntriesToAdd(elementSchema);
+          while (numEntriesToAdd > 0) {
+            listRes.add(typeConvert(elementSchema));
+            numEntriesToAdd--;
+          }
+        } else {
+          listRes.add(typeConvert(elementSchema));
+        }
+        return listRes;
+      case MAP:
+        Schema valueSchema = localSchema.getValueType();
+        Map<String, Object> mapRes = new HashMap<String, Object>();
+        if (isPrimitive(valueSchema) && this.shouldAddMore) {
+          int numEntriesToAdd = numEntriesToAdd(valueSchema);
+          while (numEntriesToAdd > 0) {
+            mapRes.put(UUID.randomUUID().toString(), typeConvert(valueSchema));
+            numEntriesToAdd--;
+          }
+        } else {
+          mapRes.put(UUID.randomUUID().toString(), typeConvert(valueSchema));
+        }
+        return mapRes;
+      case BYTES:
+        return ByteBuffer.wrap(UUID.randomUUID().toString().getBytes(Charset.defaultCharset()));
+      case FIXED:
+        return generateFixedType(localSchema);
+      default:
+        throw new IllegalArgumentException(
+            "Cannot handle type: " + localSchema.getType());
+    }
+  }
+
+  private Object generateFixedType(Schema localSchema) {
+    // TODO: Need to implement valid data generation for fixed type
+    GenericFixed genericFixed = new GenericData.Fixed(localSchema);
+    switch (localSchema.getLogicalType().getName()) {
+      case UUID_NAME:
+        ((Fixed) genericFixed).bytes(UUID.randomUUID().toString().getBytes());
+        return genericFixed;
+      case DECIMAL:
+        return genericFixed;
+      case DATE:
+        return genericFixed;
+      case TIME_MILLIS:
+        return genericFixed;
+      default:
+        throw new IllegalArgumentException(
+            "Cannot handle type: " + localSchema.getLogicalType());
+    }
+  }
+
+  public boolean validate(GenericRecord record) {
+    return genericData.validate(baseSchema, record);
+  }
+
+  protected boolean isOption(Schema schema) {
+    return schema.getType().equals(Schema.Type.UNION)
+        && schema.getTypes().size() == 2
+        && (schema.getTypes().get(0).getType().equals(Schema.Type.NULL)
+        || schema.getTypes().get(1).getType().equals(Schema.Type.NULL));
+  }
+
+  protected Schema getNonNull(Schema schema) {
+    List<Schema> types = schema.getTypes();
+    return types.get(0).getType().equals(Schema.Type.NULL) ? types.get(1) : types.get(0);
+  }
+
+  public int getEstimatedFullPayloadSize() {
+    return estimatedFullPayloadSize;
+  }
+
+  private int getSize(Schema elementSchema) {
+    switch (elementSchema.getType()) {
+      case BOOLEAN:
+        return 1;
+      case DOUBLE:
+        return Double.BYTES;
+      case FLOAT:
+        return Float.BYTES;
+      case INT:
+        return Integer.BYTES;
+      case LONG:
+        return Long.BYTES;
+      case STRING:
+        return UUID.randomUUID().toString().length();
+      case ENUM:
+        return 1;
+      case BYTES:
+        return UUID.randomUUID().toString().length();
+      case FIXED:
+        return elementSchema.getFixedSize();
+      default:
+        throw new RuntimeException("Unknown type " + elementSchema.getType());
+    }
+  }
+
+  private int numEntriesToAdd(Schema elementSchema) {
+    // Find the size of the primitive data type in bytes
+    int primitiveDataTypeSize = getSize(elementSchema);
+    int numEntriesToAdd = numberOfBytesToAdd / primitiveDataTypeSize;
+    // If more than 10 entries are being added for this same complex field and there are still more complex fields to
+    // be visited in the schema, reduce the number of entries to add by a factor of 10 to allow for other complex
+    // fields to pack some entries
+    if (numEntriesToAdd % 10 > 0 && this.numberOfComplexFields > 1) {
+      numEntriesToAdd = numEntriesToAdd / 10;
+      numberOfBytesToAdd -= numEntriesToAdd * primitiveDataTypeSize;
+      this.shouldAddMore = true;
+    } else {
+      this.numberOfBytesToAdd = 0;
+      this.shouldAddMore = false;
+    }
+    this.numberOfComplexFields -= 1;
+    return numEntriesToAdd;
+  }
+}
+
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadSizeEstimator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadSizeEstimator.java
new file mode 100644
index 0000000..c6a8f4e
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordFullPayloadSizeEstimator.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.integ.testsuite.generator;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.common.util.collection.Pair;
+
+/**
+ * This is a GenericRecord payload estimator estimates the size of a full generic record {@link GenericRecord}.
+ * A full record is defined as "Every field of a generic record created contains 1 random value"
+ */
+public class GenericRecordFullPayloadSizeEstimator implements Serializable {
+
+  private final transient Schema baseSchema;
+
+  // This variable is used to track the number of complex/collection fields with primitive data types at their leaf.
+  // This is used to figure out how many entries can be packed in such a collection field to meet the min payload
+  // size requested
+  private final transient AtomicInteger counter = new AtomicInteger(0);
+
+  public GenericRecordFullPayloadSizeEstimator(Schema schema) {
+    this.baseSchema = schema;
+  }
+
+  public Pair<Integer, Integer> typeEstimateAndNumComplexFields() {
+    int size = estimate(baseSchema);
+    return Pair.of(size, counter.get());
+  }
+
+  /**
+   * This method estimates the size of the payload if all entries of this payload were populated with one value.
+   * For eg. A primitive data type such as String will be populated with {@link UUID} so the length if 36 bytes
+   * whereas a complex data type such as an Array of type Int, will be populated with exactly 1 Integer value.
+   */
+  protected int estimate(Schema schema) {
+    long size = 0;
+    for (Schema.Field f : schema.getFields()) {
+      size += typeEstimate(f.schema());
+    }
+    return (int) size;
+  }
+
+  private long typeEstimate(Schema schema) {
+    Schema localSchema = schema;
+    if (isOption(schema)) {
+      localSchema = getNonNull(schema);
+    }
+    switch (localSchema.getType()) {
+      case BOOLEAN:
+        return 1;
+      case DOUBLE:
+        return 8;
+      case FLOAT:
+        return 4;
+      case INT:
+        return 4;
+      case LONG:
+        return 8;
+      case STRING:
+        return UUID.randomUUID().toString().length();
+      case ENUM:
+        return 1;
+      case RECORD:
+        return estimate(localSchema);
+      case ARRAY:
+        if (GenericRecordFullPayloadGenerator.isPrimitive(localSchema.getElementType())) {
+          counter.addAndGet(1);
+        }
+        Schema elementSchema = localSchema.getElementType();
+        return typeEstimate(elementSchema);
+      case MAP:
+        if (GenericRecordFullPayloadGenerator.isPrimitive(localSchema.getValueType())) {
+          counter.addAndGet(1);
+        }
+        Schema valueSchema = localSchema.getValueType();
+        return UUID.randomUUID().toString().length() + typeEstimate(valueSchema);
+      case BYTES:
+        return UUID.randomUUID().toString().length();
+      case FIXED:
+        return localSchema.getFixedSize();
+      default:
+        throw new IllegalArgumentException(
+            "Cannot handle type: " + localSchema.getType());
+    }
+  }
+
+  protected boolean isOption(Schema schema) {
+    return schema.getType().equals(Schema.Type.UNION)
+        && schema.getTypes().size() == 2
+        && (schema.getTypes().get(0).getType().equals(Schema.Type.NULL)
+        || schema.getTypes().get(1).getType().equals(Schema.Type.NULL));
+  }
+
+  protected Schema getNonNull(Schema schema) {
+    List<Schema> types = schema.getTypes();
+    return types.get(0).getType().equals(Schema.Type.NULL) ? types.get(1) : types.get(0);
+  }
+
+}
+
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordPartialPayloadGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordPartialPayloadGenerator.java
new file mode 100644
index 0000000..f7e4174
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/GenericRecordPartialPayloadGenerator.java
@@ -0,0 +1,82 @@
+/*
+ * 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.hudi.integ.testsuite.generator;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+
+/**
+ * This is a GenericRecord payload generator that generates partial generic records {@link GenericRecord}. A partial
+ * records is one that has some fields of the schema NULL or NOT PRESENT. This payload enables us to simulate
+ * creation of partial records which are possible in many cases, especially for database change logs.
+ */
+public class GenericRecordPartialPayloadGenerator extends GenericRecordFullPayloadGenerator {
+
+  public GenericRecordPartialPayloadGenerator(Schema schema) {
+    super(schema);
+  }
+
+  public GenericRecordPartialPayloadGenerator(Schema schema, int minPayloadSize) {
+    super(schema, minPayloadSize);
+  }
+
+  @Override
+  protected GenericRecord convert(Schema schema) {
+    GenericRecord record = super.convertPartial(schema);
+    return record;
+  }
+
+  private void setNull(GenericRecord record) {
+    for (Schema.Field field : record.getSchema().getFields()) {
+      // A random boolean decides whether this field of the generic record should be present or absent.
+      // Using this we can set only a handful of fields in the record and generate partial records
+      boolean setNull = random.nextBoolean();
+      if (setNull) { // TODO : DO NOT SET THE RECORD KEY FIELDS TO NULL
+        record.put(field.name(), null);
+      } else {
+        if (record.get(field.name()) instanceof GenericData.Record) {
+          setNull((GenericData.Record) record.get(field.name()));
+        }
+      }
+    }
+  }
+
+  @Override
+  public boolean validate(GenericRecord record) {
+    return validate((Object) record);
+  }
+
+  // Atleast 1 entry should be null
+  private boolean validate(Object object) {
+    if (object == null) {
+      return true;
+    } else if (object instanceof GenericRecord) {
+      for (Schema.Field field : ((GenericRecord) object).getSchema().getFields()) {
+        boolean ret = validate(((GenericRecord) object).get(field.name()));
+        if (ret) {
+          return ret;
+        }
+      }
+    }
+    return false;
+  }
+
+}
+
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/LazyRecordGeneratorIterator.java
similarity index 58%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/LazyRecordGeneratorIterator.java
index 7dde326..a775d37 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/LazyRecordGeneratorIterator.java
@@ -16,28 +16,33 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
-
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
+package org.apache.hudi.integ.testsuite.generator;
 
+import java.util.Iterator;
 import org.apache.avro.generic.GenericRecord;
-
-import java.io.Serializable;
+import org.apache.hudi.client.utils.LazyIterableIterator;
 
 /**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
+ * A lazy record generator to generate {@link GenericRecord}s lazily and not hold a list of records in memory.
  */
-public abstract class KeyGenerator implements Serializable {
+public class LazyRecordGeneratorIterator extends
+    LazyIterableIterator<GenericRecord, GenericRecord> {
 
-  protected transient TypedProperties config;
+  public LazyRecordGeneratorIterator(Iterator<GenericRecord> inputItr) {
+    super(inputItr);
+  }
 
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
+  @Override
+  protected void start() {
   }
 
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
+  @Override
+  protected GenericRecord computeNext() {
+    return inputItr.next();
+  }
+
+  @Override
+  protected void end() {
+
+  }
 }
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/UpdateGeneratorIterator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/UpdateGeneratorIterator.java
new file mode 100644
index 0000000..685a5c7
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/generator/UpdateGeneratorIterator.java
@@ -0,0 +1,56 @@
+/*
+ * 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.hudi.integ.testsuite.generator;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+
+public class UpdateGeneratorIterator implements Iterator<GenericRecord> {
+
+  // Use the full payload generator as default
+  private GenericRecordFullPayloadGenerator generator;
+  private List<String> blackListedFields;
+  // iterator
+  private Iterator<GenericRecord> itr;
+
+  public UpdateGeneratorIterator(Iterator<GenericRecord> itr, String schemaStr, List<String> partitionPathFieldNames,
+      List<String> recordKeyFieldNames, int minPayloadSize) {
+    this.itr = itr;
+    this.blackListedFields = new ArrayList<>();
+    this.blackListedFields.addAll(partitionPathFieldNames);
+    this.blackListedFields.addAll(recordKeyFieldNames);
+    Schema schema = new Schema.Parser().parse(schemaStr);
+    this.generator = new GenericRecordFullPayloadGenerator(schema, minPayloadSize);
+  }
+
+  @Override
+  public boolean hasNext() {
+    return itr.hasNext();
+  }
+
+  @Override
+  public GenericRecord next() {
+    GenericRecord newRecord = itr.next();
+    return this.generator.randomize(newRecord, this.blackListedFields);
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/DFSTestSuitePathSelector.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/DFSTestSuitePathSelector.java
new file mode 100644
index 0000000..b67e21f
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/DFSTestSuitePathSelector.java
@@ -0,0 +1,92 @@
+/*
+ * 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.hudi.integ.testsuite.helpers;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.utilities.sources.helpers.DFSPathSelector;
+
+/**
+ * A custom dfs path selector used only for the hudi test suite. To be used only if workload is not run inline.
+ */
+public class DFSTestSuitePathSelector extends DFSPathSelector {
+
+  public DFSTestSuitePathSelector(TypedProperties props, Configuration hadoopConf) {
+    super(props, hadoopConf);
+  }
+
+  @Override
+  public Pair<Option<String>, String> getNextFilePathsAndMaxModificationTime(
+      Option<String> lastCheckpointStr, long sourceLimit) {
+
+    Integer lastBatchId;
+    Integer nextBatchId;
+    try {
+      if (lastCheckpointStr.isPresent()) {
+        lastBatchId = Integer.parseInt(lastCheckpointStr.get());
+        nextBatchId = lastBatchId + 1;
+      } else {
+        lastBatchId = -1;
+        nextBatchId = 0;
+      }
+      // obtain all eligible files for the batch
+      List<FileStatus> eligibleFiles = new ArrayList<>();
+      FileStatus[] fileStatuses = fs.globStatus(
+          new Path(props.getString(Config.ROOT_INPUT_PATH_PROP), "*"));
+      for (FileStatus fileStatus : fileStatuses) {
+        if (!fileStatus.isDirectory() || IGNORE_FILEPREFIX_LIST.stream()
+            .anyMatch(pfx -> fileStatus.getPath().getName().startsWith(pfx))) {
+          continue;
+        } else if (fileStatus.getPath().getName().compareTo(lastBatchId.toString()) > 0 && fileStatus.getPath()
+            .getName().compareTo(nextBatchId.toString()) <= 0) {
+          RemoteIterator<LocatedFileStatus> files = fs.listFiles(fileStatus.getPath(), true);
+          while (files.hasNext()) {
+            eligibleFiles.add(files.next());
+          }
+        }
+      }
+      // no data to readAvro
+      if (eligibleFiles.size() == 0) {
+        return new ImmutablePair<>(Option.empty(),
+            lastCheckpointStr.orElseGet(() -> String.valueOf(Long.MIN_VALUE)));
+      }
+      // readAvro the files out.
+      String pathStr = eligibleFiles.stream().map(f -> f.getPath().toString())
+          .collect(Collectors.joining(","));
+
+      return new ImmutablePair<>(Option.ofNullable(pathStr), String.valueOf(nextBatchId));
+    } catch (IOException ioe) {
+      throw new HoodieIOException(
+          "Unable to readAvro from source from checkpoint: " + lastCheckpointStr, ioe);
+    }
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java
new file mode 100644
index 0000000..88022ed
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java
@@ -0,0 +1,74 @@
+/*
+ * 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.hudi.integ.testsuite.helpers;
+
+import java.io.IOException;
+import java.net.BindException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hive.service.server.HiveServer2;
+import org.apache.hudi.hive.testutils.HiveTestService;
+import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.thrift.transport.TTransportException;
+
+/**
+ * Hive Service provider.
+ */
+public class HiveServiceProvider {
+
+  private HiveTestService hiveService;
+  private HiveServer2 hiveServer;
+  private Config config;
+
+  private static final Logger LOG = LogManager.getLogger(HiveServiceProvider.class);
+
+  public HiveServiceProvider(Config config) {
+    this.config = config;
+  }
+
+  public void startLocalHiveServiceIfNeeded(Configuration configuration) throws IOException {
+    if (config.isHiveLocal()) {
+      hiveService = new HiveTestService(configuration);
+      hiveServer = hiveService.start();
+    }
+  }
+
+  public void syncToLocalHiveIfNeeded(HoodieTestSuiteWriter writer) {
+    if (this.config.isHiveLocal()) {
+      writer.getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync()
+          .syncHive(getLocalHiveServer().getHiveConf());
+    } else {
+      writer.getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync().syncHive();
+    }
+  }
+
+  public void stopLocalHiveServiceIfNeeded() throws IOException {
+    if (config.isHiveLocal()) {
+      if (hiveService != null) {
+        hiveService.stop();
+      }
+    }
+  }
+
+  public HiveServer2 getLocalHiveServer() {
+    return hiveServer;
+  }
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSAvroDeltaInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSAvroDeltaInputReader.java
new file mode 100644
index 0000000..f1bb02a
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSAvroDeltaInputReader.java
@@ -0,0 +1,89 @@
+/*
+ * 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.hudi.integ.testsuite.reader;
+
+import java.io.IOException;
+import java.util.Arrays;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.writer.AvroFileDeltaInputWriter;
+import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.SparkSession;
+
+/**
+ * A reader of {@link DeltaOutputMode#DFS} and {@link DeltaInputType#AVRO}.
+ */
+public class DFSAvroDeltaInputReader extends DFSDeltaInputReader {
+
+  private final SparkSession sparkSession;
+  private final String schemaStr;
+  private final String basePath;
+  private final Option<String> structName;
+  private final Option<String> nameSpace;
+  protected PathFilter filter = (path) -> {
+    if (path.toUri().toString().contains(AvroFileDeltaInputWriter.AVRO_EXTENSION)) {
+      return true;
+    } else {
+      return false;
+    }
+  };
+
+  public DFSAvroDeltaInputReader(SparkSession sparkSession, String schemaStr, String basePath,
+      Option<String> structName,
+      Option<String> nameSpace) {
+    this.sparkSession = sparkSession;
+    this.schemaStr = schemaStr;
+    this.basePath = basePath;
+    this.structName = structName;
+    this.nameSpace = nameSpace;
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> read(long totalRecordsToRead) throws IOException {
+    return SparkBasedReader.readAvro(sparkSession, schemaStr, getFilePathsToRead(basePath, filter, totalRecordsToRead),
+        structName, nameSpace);
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> read(int numPartitions, long approxNumRecords) throws IOException {
+    throw new UnsupportedOperationException("cannot generate updates");
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> read(int numPartitions, int numFiles, long approxNumRecords) throws IOException {
+    throw new UnsupportedOperationException("cannot generate updates");
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> read(int numPartitions, int numFiles, double percentageRecordsPerFile)
+      throws IOException {
+    throw new UnsupportedOperationException("cannot generate updates");
+  }
+
+  @Override
+  protected long analyzeSingleFile(String filePath) {
+    JavaRDD<GenericRecord> recordsFromOneFile = SparkBasedReader
+        .readAvro(sparkSession, schemaStr, Arrays.asList(filePath),
+            structName, nameSpace);
+    return recordsFromOneFile.count();
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSDeltaInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSDeltaInputReader.java
new file mode 100644
index 0000000..da92681
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSDeltaInputReader.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.integ.testsuite.reader;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.collection.Pair;
+
+/**
+ * This class helps to estimate the number of files to read a given number of total records.
+ * Use this class for all DFS based implementations of {@link DeltaInputReader}
+ */
+public abstract class DFSDeltaInputReader implements DeltaInputReader<GenericRecord> {
+
+  protected List<String> getFilePathsToRead(String basePath, PathFilter filter, long totalRecordsToRead) throws
+      IOException {
+    FileSystem fs = FSUtils.getFs(basePath, new Configuration());
+    // TODO : Sort list by file size and take the median file status to ensure fair calculation and change to remote
+    // iterator
+    List<FileStatus> fileStatuses = Arrays.asList(fs.globStatus(new Path(basePath, "*/*"), filter));
+    if (fileStatuses.size() > 0) {
+      FileStatus status = fileStatuses.get(0);
+      long avgNumRecordsPerFile = analyzeSingleFile(status.getPath().toString());
+      long numFilesToMatchExpectedRecords = (long) Math.ceil((double) totalRecordsToRead / (double)
+          avgNumRecordsPerFile);
+      long avgSizeOfEachFile = status.getLen();
+      long totalSizeToRead = avgSizeOfEachFile * numFilesToMatchExpectedRecords;
+      // choose N files with that length
+      Pair<Integer, Integer> fileStatusIndexRange = getFileStatusIndexRange(fileStatuses, avgSizeOfEachFile,
+          totalSizeToRead);
+      int startIndex = fileStatusIndexRange.getLeft();
+      List<String> filePaths = new ArrayList<>();
+      while (startIndex == 0 || startIndex < fileStatusIndexRange.getRight()) {
+        filePaths.add(fileStatuses.get(startIndex).getPath().toString());
+        startIndex++;
+      }
+      return filePaths;
+    }
+    return Collections.emptyList();
+  }
+
+  protected Pair<Integer, Integer> getFileStatusIndexRange(List<FileStatus> fileStatuses, long averageFileSize, long
+      totalSizeToRead) {
+    long totalSizeOfFilesPresent = 0;
+    int startOffset = 0;
+    int endOffset = 0;
+    for (FileStatus fileStatus : fileStatuses) {
+      // If current file length is greater than averageFileSize, increment by averageFileSize since our
+      // totalSizeToRead calculation is based on the averageRecordSize * numRecordsToRead.
+      if (fileStatus.getLen() > averageFileSize) {
+        totalSizeOfFilesPresent += averageFileSize;
+      } else {
+        totalSizeOfFilesPresent += fileStatus.getLen();
+      }
+      if (totalSizeOfFilesPresent <= totalSizeToRead) {
+        endOffset++;
+        continue;
+      } else {
+        return Pair.of(startOffset, endOffset);
+      }
+    }
+    return Pair.of(startOffset, endOffset);
+  }
+
+  /**
+   * Implementation of {@link DeltaInputReader}s to provide a way to read a single file on DFS and provide an
+   * average number of records across N files.
+   */
+  protected long analyzeSingleFile(String filePath) {
+    throw new UnsupportedOperationException("No implementation found");
+  }
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java
new file mode 100644
index 0000000..209aa46
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java
@@ -0,0 +1,338 @@
+/*
+ * 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.hudi.integ.testsuite.reader;
+
+import static java.util.Map.Entry.comparingByValue;
+import static java.util.stream.Collectors.toMap;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
+import org.apache.hudi.common.table.view.TableFileSystemView;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ParquetReaderIterator;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieMemoryConfig;
+import org.apache.parquet.avro.AvroParquetReader;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Tuple2;
+
+/**
+ * This class helps to generate updates from an already existing hoodie dataset. It supports generating updates in
+ * across partitions, files and records.
+ */
+public class DFSHoodieDatasetInputReader extends DFSDeltaInputReader {
+
+  private static Logger log = LoggerFactory.getLogger(DFSHoodieDatasetInputReader.class);
+
+  private transient JavaSparkContext jsc;
+  private String schemaStr;
+  private HoodieTableMetaClient metaClient;
+
+  public DFSHoodieDatasetInputReader(JavaSparkContext jsc, String basePath, String schemaStr) {
+    this.jsc = jsc;
+    this.schemaStr = schemaStr;
+    this.metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
+  }
+
+  protected List<String> getPartitions(Option<Integer> partitionsLimit) throws IOException {
+    List<String> partitionPaths = FSUtils
+        .getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), false);
+    // Sort partition so we can pick last N partitions by default
+    Collections.sort(partitionPaths);
+    if (!partitionPaths.isEmpty()) {
+      ValidationUtils.checkArgument(partitionPaths.size() >= partitionsLimit.get(),
+          "Cannot generate updates for more partitions " + "than present in the dataset, partitions "
+              + "requested " + partitionsLimit.get() + ", partitions present " + partitionPaths.size());
+      return partitionPaths.subList(0, partitionsLimit.get());
+    }
+    return partitionPaths;
+
+  }
+
+  private JavaPairRDD<String, Iterator<FileSlice>> getPartitionToFileSlice(HoodieTableMetaClient metaClient,
+      List<String> partitionPaths) {
+    TableFileSystemView.SliceView fileSystemView = new HoodieTableFileSystemView(metaClient,
+        metaClient.getCommitsAndCompactionTimeline().filterCompletedInstants());
+    // pass num partitions to another method
+    JavaPairRDD<String, Iterator<FileSlice>> partitionToFileSliceList = jsc.parallelize(partitionPaths).mapToPair(p -> {
+      return new Tuple2<>(p, fileSystemView.getLatestFileSlices(p).iterator());
+    });
+    return partitionToFileSliceList;
+  }
+
+  @Override
+  protected long analyzeSingleFile(String filePath) {
+    return SparkBasedReader.readParquet(new SparkSession(jsc.sc()), Arrays.asList(filePath),
+        Option.empty(), Option.empty()).count();
+  }
+
+  private JavaRDD<GenericRecord> fetchAnyRecordsFromDataset(Option<Long> numRecordsToUpdate) throws IOException {
+    return fetchRecordsFromDataset(Option.empty(), Option.empty(), numRecordsToUpdate, Option.empty());
+  }
+
+  private JavaRDD<GenericRecord> fetchAnyRecordsFromDataset(Option<Long> numRecordsToUpdate, Option<Integer>
+      numPartitions) throws IOException {
+    return fetchRecordsFromDataset(numPartitions, Option.empty(), numRecordsToUpdate, Option.empty());
+  }
+
+  private JavaRDD<GenericRecord> fetchPercentageRecordsFromDataset(Option<Integer> numPartitions, Option<Integer>
+      numFiles, Option<Double> percentageRecordsPerFile) throws IOException {
+    return fetchRecordsFromDataset(numPartitions, numFiles, Option.empty(), percentageRecordsPerFile);
+  }
+
+  private JavaRDD<GenericRecord> fetchRecordsFromDataset(Option<Integer> numPartitions, Option<Integer>
+      numFiles, Option<Long> numRecordsToUpdate) throws IOException {
+    return fetchRecordsFromDataset(numPartitions, numFiles, numRecordsToUpdate, Option.empty());
+  }
+
+  private JavaRDD<GenericRecord> fetchRecordsFromDataset(Option<Integer> numPartitions, Option<Integer> numFiles,
+      Option<Long> numRecordsToUpdate, Option<Double> percentageRecordsPerFile) throws IOException {
+    log.info("NumPartitions : {}, NumFiles : {}, numRecordsToUpdate : {}, percentageRecordsPerFile : {}",
+        numPartitions, numFiles, numRecordsToUpdate, percentageRecordsPerFile);
+    List<String> partitionPaths = getPartitions(numPartitions);
+    // Read all file slices in the partition
+    JavaPairRDD<String, Iterator<FileSlice>> partitionToFileSlice = getPartitionToFileSlice(metaClient,
+        partitionPaths);
+    // TODO : read record count from metadata
+    // Read the records in a single file
+    long recordsInSingleFile = iteratorSize(readParquetOrLogFiles(getSingleSliceFromRDD(partitionToFileSlice)));
+    int numFilesToUpdate;
+    long numRecordsToUpdatePerFile;
+    if (!numFiles.isPresent() || numFiles.get() == 0) {
+      // If num files are not passed, find the number of files to update based on total records to update and records
+      // per file
+      numFilesToUpdate = (int) (numRecordsToUpdate.get() / recordsInSingleFile);
+      log.info("Files to update {}", numFilesToUpdate);
+      numRecordsToUpdatePerFile = recordsInSingleFile;
+    } else {
+      // If num files is passed, find the number of records per file based on either percentage or total records to
+      // update and num files passed
+      numFilesToUpdate = numFiles.get();
+      numRecordsToUpdatePerFile = percentageRecordsPerFile.isPresent() ? (long) (recordsInSingleFile
+          * percentageRecordsPerFile.get()) : numRecordsToUpdate.get() / numFilesToUpdate;
+    }
+    // Adjust the number of files to read per partition based on the requested partition & file counts
+    Map<String, Integer> adjustedPartitionToFileIdCountMap = getFilesToReadPerPartition(partitionToFileSlice,
+        getPartitions(numPartitions).size(), numFilesToUpdate);
+    JavaRDD<GenericRecord> updates = projectSchema(generateUpdates(adjustedPartitionToFileIdCountMap,
+        partitionToFileSlice, numFilesToUpdate, (int) numRecordsToUpdatePerFile));
+    if (numRecordsToUpdate.isPresent() && numFiles.isPresent() && numFiles.get() != 0 && numRecordsToUpdate.get()
+        != numRecordsToUpdatePerFile * numFiles.get()) {
+      long remainingRecordsToAdd = (numRecordsToUpdate.get() - (numRecordsToUpdatePerFile * numFiles.get()));
+      updates = updates.union(projectSchema(jsc.parallelize(generateUpdates(adjustedPartitionToFileIdCountMap,
+          partitionToFileSlice, numFilesToUpdate, (int) remainingRecordsToAdd).take((int) remainingRecordsToAdd))));
+    }
+    log.info("Finished generating updates");
+    return updates;
+  }
+
+  private JavaRDD<GenericRecord> projectSchema(JavaRDD<GenericRecord> updates) {
+    // The records read from the hoodie dataset have the hoodie record fields, rewrite the record to eliminate them
+    return updates
+        .map(r -> HoodieAvroUtils.rewriteRecordWithOnlyNewSchemaFields(r, new Schema.Parser().parse(schemaStr)));
+  }
+
+  private JavaRDD<GenericRecord> generateUpdates(Map<String, Integer> adjustedPartitionToFileIdCountMap,
+      JavaPairRDD<String, Iterator<FileSlice>> partitionToFileSlice, int numFiles, int numRecordsToReadPerFile) {
+    return partitionToFileSlice.map(p -> {
+      int maxFilesToRead = adjustedPartitionToFileIdCountMap.get(p._1);
+      return iteratorLimit(p._2, maxFilesToRead);
+    }).flatMap(p -> p).repartition(numFiles).map(fileSlice -> {
+      if (numRecordsToReadPerFile > 0) {
+        return iteratorLimit(readParquetOrLogFiles(fileSlice), numRecordsToReadPerFile);
+      } else {
+        return readParquetOrLogFiles(fileSlice);
+      }
+    }).flatMap(p -> p).map(i -> (GenericRecord) i);
+  }
+
+  private Map<String, Integer> getFilesToReadPerPartition(JavaPairRDD<String, Iterator<FileSlice>>
+      partitionToFileSlice, Integer numPartitions, Integer numFiles) {
+    int numFilesPerPartition = (int) Math.ceil(numFiles / numPartitions);
+    Map<String, Integer> partitionToFileIdCountMap = partitionToFileSlice
+        .mapToPair(p -> new Tuple2<>(p._1, iteratorSize(p._2))).collectAsMap();
+    long totalExistingFilesCount = partitionToFileIdCountMap.values().stream().reduce((a, b) -> a + b).get();
+    ValidationUtils.checkArgument(totalExistingFilesCount >= numFiles, "Cannot generate updates "
+        + "for more files than present in the dataset, file requested " + numFiles + ", files present "
+        + totalExistingFilesCount);
+    Map<String, Integer> partitionToFileIdCountSortedMap = partitionToFileIdCountMap
+        .entrySet()
+        .stream()
+        .sorted(comparingByValue())
+        .collect(toMap(Map.Entry::getKey, Map.Entry::getValue, (e1, e2) -> e2,
+            LinkedHashMap::new));
+    // Limit files to be read per partition
+    Map<String, Integer> adjustedPartitionToFileIdCountMap = new HashMap<>();
+    partitionToFileIdCountSortedMap.entrySet().stream().forEach(e -> {
+      if (e.getValue() <= numFilesPerPartition) {
+        adjustedPartitionToFileIdCountMap.put(e.getKey(), e.getValue());
+      } else {
+        adjustedPartitionToFileIdCountMap.put(e.getKey(), numFilesPerPartition);
+      }
+    });
+    return adjustedPartitionToFileIdCountMap;
+  }
+
+  private FileSlice getSingleSliceFromRDD(JavaPairRDD<String, Iterator<FileSlice>> partitionToFileSlice) {
+    return partitionToFileSlice.map(f -> {
+      FileSlice slice = f._2.next();
+      FileSlice newSlice = new FileSlice(slice.getFileGroupId(), slice.getBaseInstantTime());
+      if (slice.getBaseFile().isPresent()) {
+        newSlice.setBaseFile(slice.getBaseFile().get());
+      } else {
+        slice.getLogFiles().forEach(l -> {
+          newSlice.addLogFile(l);
+        });
+      }
+      return newSlice;
+    }).take(1).get(0);
+  }
+
+  private Iterator<IndexedRecord> readParquetOrLogFiles(FileSlice fileSlice) throws IOException {
+    if (fileSlice.getBaseFile().isPresent()) {
+      Iterator<IndexedRecord> itr =
+          new ParquetReaderIterator<IndexedRecord>(AvroParquetReader.<IndexedRecord>builder(new
+              Path(fileSlice.getBaseFile().get().getPath())).withConf(metaClient.getHadoopConf()).build());
+      return itr;
+    } else {
+      // If there is no data file, fall back to reading log files
+      HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(metaClient.getFs(),
+          metaClient.getBasePath(),
+          fileSlice.getLogFiles().map(l -> l.getPath().getName()).collect(Collectors.toList()),
+          new Schema.Parser().parse(schemaStr), metaClient.getActiveTimeline().getCommitsTimeline()
+          .filterCompletedInstants().lastInstant().get().getTimestamp(),
+          HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES, true, false,
+          HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE,
+          HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH);
+      // readAvro log files
+      Iterable<HoodieRecord<? extends HoodieRecordPayload>> iterable = () -> scanner.iterator();
+      Schema schema = new Schema.Parser().parse(schemaStr);
+      return StreamSupport.stream(iterable.spliterator(), false)
+          .map(e -> {
+            try {
+              return (IndexedRecord) e.getData().getInsertValue(schema).get();
+            } catch (IOException io) {
+              throw new UncheckedIOException(io);
+            }
+          }).iterator();
+    }
+  }
+
+  /**
+   * Returns the number of elements remaining in {@code iterator}. The iterator
+   * will be left exhausted: its {@code hasNext()} method will return
+   * {@code false}.
+   */
+  private static int iteratorSize(Iterator<?> iterator) {
+    int count = 0;
+    while (iterator.hasNext()) {
+      iterator.next();
+      count++;
+    }
+    return count;
+  }
+
+  /**
+   * Creates an iterator returning the first {@code limitSize} elements of the
+   * given iterator. If the original iterator does not contain that many
+   * elements, the returned iterator will have the same behavior as the original
+   * iterator. The returned iterator supports {@code remove()} if the original
+   * iterator does.
+   *
+   * @param iterator the iterator to limit
+   * @param limitSize the maximum number of elements in the returned iterator
+   * @throws IllegalArgumentException if {@code limitSize} is negative
+   */
+  private static <T> Iterator<T> iteratorLimit(
+      final Iterator<T> iterator, final int limitSize) {
+    ValidationUtils.checkArgument(iterator != null, "iterator is null");
+    ValidationUtils.checkArgument(limitSize >= 0, "limit is negative");
+    return new Iterator<T>() {
+      private int count;
+
+      @Override
+      public boolean hasNext() {
+        return count < limitSize && iterator.hasNext();
+      }
+
+      @Override
+      public T next() {
+        if (!hasNext()) {
+          throw new NoSuchElementException();
+        }
+        count++;
+        return iterator.next();
+      }
+
+      @Override
+      public void remove() {
+        iterator.remove();
+      }
+    };
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> read(long numRecords) throws IOException {
+    return fetchAnyRecordsFromDataset(Option.of(numRecords));
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> read(int numPartitions, long approxNumRecords) throws IOException {
+    return fetchAnyRecordsFromDataset(Option.of(approxNumRecords), Option.of(numPartitions));
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> read(int numPartitions, int numFiles, long numRecords) throws IOException {
+    return fetchRecordsFromDataset(Option.of(numPartitions), Option.of(numFiles), Option.of(numRecords));
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> read(int numPartitions, int numFiles, double percentageRecordsPerFile)
+      throws IOException {
+    return fetchPercentageRecordsFromDataset(Option.of(numPartitions), Option.of(numFiles),
+        Option.of(percentageRecordsPerFile));
+  }
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DeltaInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DeltaInputReader.java
new file mode 100644
index 0000000..3ba4041
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DeltaInputReader.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.hudi.integ.testsuite.reader;
+
+import java.io.IOException;
+import java.io.Serializable;
+import org.apache.spark.api.java.JavaRDD;
+
+/**
+ * Implementations of {@link DeltaInputReader} will read the configured input type and provide an RDD of records to the
+ * client.
+ *
+ * @param <O> Read result data type
+ */
+public interface DeltaInputReader<O> extends Serializable {
+
+  /**
+   * Attempts to reads an approximate number of records close to approxNumRecords.
+   * This highly depends on the number of records already present in the input.
+   */
+  JavaRDD<O> read(long approxNumRecords) throws IOException;
+
+  /**
+   * @throws IOException Attempts to read approx number of records (exact if equal or more records available)
+   * across requested number of
+   * partitions.
+   */
+  JavaRDD<O> read(int numPartitions, long approxNumRecords) throws IOException;
+
+  /**
+   * @throws IOException Attempts to read approx number of records (exact if equal or more records available)
+   * across requested number of
+   * partitions and number of files.
+   * 1. Find numFiles across numPartitions
+   * 2. numRecordsToReadPerFile = approxNumRecords / numFiles
+   */
+  JavaRDD<O> read(int numPartitions, int numFiles, long approxNumRecords) throws IOException;
+
+  /**
+   * @throws IOException Attempts to a % of records per file across requested number of partitions and number of files.
+   * 1. Find numFiles across numPartitions
+   * 2. numRecordsToReadPerFile = approxNumRecordsPerFile * percentageRecordsPerFile
+   */
+  JavaRDD<O> read(int numPartitions, int numFiles, double percentageRecordsPerFile) throws IOException;
+
+}
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DeltaInputType.java
similarity index 56%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DeltaInputType.java
index 7dde326..137a503 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DeltaInputType.java
@@ -16,28 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
-
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
-
-import org.apache.avro.generic.GenericRecord;
-
-import java.io.Serializable;
+package org.apache.hudi.integ.testsuite.reader;
 
 /**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
+ * Supported delta input data types.
  */
-public abstract class KeyGenerator implements Serializable {
-
-  protected transient TypedProperties config;
-
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
-  }
-
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
+public enum DeltaInputType {
+  AVRO, PARQUET
 }
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/SparkBasedReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/SparkBasedReader.java
new file mode 100644
index 0000000..16a5259
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/SparkBasedReader.java
@@ -0,0 +1,70 @@
+/*
+ * 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.hudi.integ.testsuite.reader;
+
+import java.util.List;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.utilities.schema.RowBasedSchemaProvider;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import scala.collection.JavaConverters;
+
+
+/**
+ * Helper class to read avro and/or parquet files and generate a RDD of {@link GenericRecord}.
+ */
+public class SparkBasedReader {
+
+  public static final String SPARK_AVRO_FORMAT = "avro";
+  public static final String SPARK_PARQUET_FORMAT = "com.databricks.spark.parquet";
+  private static final String AVRO_SCHEMA_OPTION_KEY = "avroSchema";
+
+  // Spark anyways globs the path and gets all the paths in memory so take the List<filePaths> as an argument.
+  // https://github.com/apache/spark/.../org/apache/spark/sql/execution/datasources/DataSource.scala#L251
+  public static JavaRDD<GenericRecord> readAvro(SparkSession sparkSession, String schemaStr, List<String> listOfPaths,
+      Option<String> structName, Option<String> nameSpace) {
+
+    Dataset<Row> dataSet = sparkSession.read()
+        .format(SPARK_AVRO_FORMAT)
+        .option(AVRO_SCHEMA_OPTION_KEY, schemaStr)
+        .load(JavaConverters.asScalaIteratorConverter(listOfPaths.iterator()).asScala().toSeq());
+
+    return AvroConversionUtils
+        .createRdd(dataSet.toDF(), structName.orElse(RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME),
+            nameSpace.orElse(RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE))
+        .toJavaRDD();
+  }
+
+  public static JavaRDD<GenericRecord> readParquet(SparkSession sparkSession, List<String>
+      listOfPaths, Option<String> structName, Option<String> nameSpace) {
+
+    Dataset<Row> dataSet = sparkSession.read()
+        .parquet((JavaConverters.asScalaIteratorConverter(listOfPaths.iterator()).asScala().toSeq()));
+
+    return AvroConversionUtils
+        .createRdd(dataSet.toDF(), structName.orElse(RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME),
+            RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE)
+        .toJavaRDD();
+  }
+
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/AvroFileDeltaInputWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/AvroFileDeltaInputWriter.java
new file mode 100644
index 0000000..2418152
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/AvroFileDeltaInputWriter.java
@@ -0,0 +1,117 @@
+/*
+ * 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.hudi.integ.testsuite.writer;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.UUID;
+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.avro.generic.IndexedRecord;
+import org.apache.avro.io.DatumWriter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation of {@link DeltaInputWriter} that writes avro records to the result file.
+ */
+public class AvroFileDeltaInputWriter implements DeltaInputWriter<GenericRecord> {
+
+  public static final String AVRO_EXTENSION = ".avro";
+  private static Logger log = LoggerFactory.getLogger(AvroFileDeltaInputWriter.class);
+  // The maximum file size for an avro file before being rolled over to a new one
+  private final Long maxFileSize;
+  private final Configuration configuration;
+  private HoodieWrapperFileSystem fs;
+  // Path of the actual avro file
+  private Path file;
+  // Base input path to write avro files under
+  // TODO : Make this bucketed so don't have a large number of files in a single directory
+  private String basePath;
+  private DatumWriter<IndexedRecord> writer;
+  private DataFileWriter<IndexedRecord> dataFileWriter;
+  private OutputStream output;
+  private Schema schema;
+  private DeltaWriteStats deltaWriteStats;
+  private long recordsWritten = 0;
+
+  // TODO : Handle failure case which may leave behind tons of small corrupt files
+  public AvroFileDeltaInputWriter(Configuration configuration, String basePath, String schemaStr, Long maxFileSize)
+      throws IOException {
+    this.schema = Schema.parse(schemaStr);
+    this.maxFileSize = maxFileSize;
+    this.configuration = configuration;
+    this.basePath = basePath;
+    Path path = new Path(basePath, new Path(UUID.randomUUID().toString() + AVRO_EXTENSION));
+    this.file = HoodieWrapperFileSystem.convertToHoodiePath(path, configuration);
+    this.fs = (HoodieWrapperFileSystem) this.file
+        .getFileSystem(FSUtils.registerFileSystem(path, configuration));
+    this.output = this.fs.create(this.file);
+    this.writer = new GenericDatumWriter(schema);
+    this.dataFileWriter = new DataFileWriter<>(writer).create(schema, output);
+    this.deltaWriteStats = new DeltaWriteStats();
+  }
+
+  @Override
+  public void writeData(GenericRecord iData) throws IOException {
+    this.dataFileWriter.append(iData);
+    recordsWritten++;
+  }
+
+  @Override
+  public boolean canWrite() {
+    return fs.getBytesWritten(file) < maxFileSize;
+  }
+
+  @Override
+  public void close() throws IOException {
+    this.deltaWriteStats.setBytesWritten(this.fs.getBytesWritten(this.file));
+    this.deltaWriteStats.setRecordsWritten(this.recordsWritten);
+    this.deltaWriteStats.setFilePath(this.file.toUri().getPath());
+    this.dataFileWriter.close();
+    log.info("New Avro File : {}", getPath());
+  }
+
+  @Override
+  public DeltaInputWriter getNewWriter() throws IOException {
+    AvroFileDeltaInputWriter avroFileDeltaInputWriter = new AvroFileDeltaInputWriter(this.configuration, this.basePath, this
+        .schema.toString(), this.maxFileSize);
+    return avroFileDeltaInputWriter;
+  }
+
+  public FileSystem getFs() {
+    return fs;
+  }
+
+  public Path getPath() {
+    return this.file;
+  }
+
+  @Override
+  public DeltaWriteStats getDeltaWriteStats() {
+    return this.deltaWriteStats;
+  }
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java
new file mode 100644
index 0000000..4c70ac5
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.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.hudi.integ.testsuite.writer;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.avro.generic.GenericRecord;
+
+/**
+ * {@link org.apache.hadoop.hdfs.DistributedFileSystem} (or {@link org.apache.hadoop.fs.LocalFileSystem}) based delta
+ * generator.
+ */
+public class DFSDeltaWriterAdapter implements DeltaWriterAdapter<GenericRecord> {
+
+  private DeltaInputWriter deltaInputGenerator;
+  private List<DeltaWriteStats> metrics = new ArrayList<>();
+
+  public DFSDeltaWriterAdapter(DeltaInputWriter<GenericRecord> deltaInputGenerator) {
+    this.deltaInputGenerator = deltaInputGenerator;
+  }
+
+  @Override
+  public List<DeltaWriteStats> write(Iterator<GenericRecord> input) throws IOException {
+    while (input.hasNext()) {
+      if (this.deltaInputGenerator.canWrite()) {
+        this.deltaInputGenerator.writeData(input.next());
+      } else if (input.hasNext()) {
+        rollOver();
+      }
+    }
+    close();
+    return this.metrics;
+  }
+
+  public void rollOver() throws IOException {
+    close();
+    this.deltaInputGenerator = this.deltaInputGenerator.getNewWriter();
+  }
+
+  private void close() throws IOException {
+    this.deltaInputGenerator.close();
+    this.metrics.add(this.deltaInputGenerator.getDeltaWriteStats());
+  }
+}
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSSparkAvroDeltaWriter.java
similarity index 53%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSSparkAvroDeltaWriter.java
index 7dde326..74cbe5d 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSSparkAvroDeltaWriter.java
@@ -16,28 +16,27 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
-
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
+package org.apache.hudi.integ.testsuite.writer;
 
+import java.io.IOException;
 import org.apache.avro.generic.GenericRecord;
-
-import java.io.Serializable;
+import org.apache.hudi.integ.testsuite.writer.DeltaWriterAdapter.SparkBasedDeltaWriter;
+import org.apache.spark.api.java.JavaRDD;
 
 /**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
+ * NEED TO IMPLEMENT A CUSTOM SPARK PARTITIONER TO ENSURE WE WRITE LARGE ENOUGH AVRO FILES.
  */
-public abstract class KeyGenerator implements Serializable {
+public class DFSSparkAvroDeltaWriter implements SparkBasedDeltaWriter<JavaRDD<GenericRecord>> {
 
-  protected transient TypedProperties config;
+  private DeltaInputWriter<JavaRDD<GenericRecord>> deltaInputWriter;
 
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
+  public DFSSparkAvroDeltaWriter(DeltaInputWriter<JavaRDD<GenericRecord>> deltaInputWriter) {
+    this.deltaInputWriter = deltaInputWriter;
   }
 
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
+  @Override
+  public JavaRDD<DeltaWriteStats> write(JavaRDD<GenericRecord> input) throws IOException {
+    this.deltaInputWriter.writeData(input);
+    return null;
+  }
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaInputWriter.java
similarity index 54%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaInputWriter.java
index 7dde326..da52434 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaInputWriter.java
@@ -16,28 +16,38 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
+package org.apache.hudi.integ.testsuite.writer;
 
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
-
-import org.apache.avro.generic.GenericRecord;
-
-import java.io.Serializable;
+import java.io.Closeable;
+import java.io.IOException;
 
 /**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
+ * Implementations of {@link DeltaInputWriter} will be able to generate data.
+ *
+ * @param <I> Data type to be generated.
  */
-public abstract class KeyGenerator implements Serializable {
+public interface DeltaInputWriter<I> extends Closeable {
 
-  protected transient TypedProperties config;
+  /**
+   * Generate any type of data.
+   */
+  void writeData(I iData) throws IOException;
 
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
-  }
+  /**
+   * Check whether more data can/should be written.
+   */
+  boolean canWrite();
 
   /**
-   * Generate a Hoodie Key out of provided generic record.
+   * Return the statistics of data written.
    */
-  public abstract HoodieKey getKey(GenericRecord record);
-}
+  DeltaWriteStats getDeltaWriteStats();
+
+  /**
+   * Return a new instance of this writer.
+   * @return
+   * @throws IOException
+   */
+  DeltaInputWriter getNewWriter() throws IOException;
+
+}
\ No newline at end of file
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaOutputMode.java
similarity index 56%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaOutputMode.java
index 7dde326..daed0fb 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaOutputMode.java
@@ -16,28 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
-
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
-
-import org.apache.avro.generic.GenericRecord;
-
-import java.io.Serializable;
+package org.apache.hudi.integ.testsuite.writer;
 
 /**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
+ * Supported output destination types for the generated delta workload.
  */
-public abstract class KeyGenerator implements Serializable {
-
-  protected transient TypedProperties config;
-
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
-  }
-
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
+public enum DeltaOutputMode {
+  KAFKA, DFS
 }
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriteStats.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriteStats.java
new file mode 100644
index 0000000..8ccd69f
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriteStats.java
@@ -0,0 +1,72 @@
+/*
+ * 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.hudi.integ.testsuite.writer;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hudi.common.util.collection.Pair;
+
+/**
+ * This class holds the write statistics for {@link DeltaInputWriter}.
+ */
+public class DeltaWriteStats implements Serializable {
+
+  // The file path (if any) for the data written
+  private String filePath;
+  // Number of bytes written before being closed
+  private long bytesWritten;
+  // Number of records written before being closed
+  private long recordsWritten;
+
+  private List<Pair<String, String>> partitionPathRecordKey = new ArrayList<>();
+
+  public String getFilePath() {
+    return filePath;
+  }
+
+  public void setFilePath(String filePath) {
+    this.filePath = filePath;
+  }
+
+  public long getBytesWritten() {
+    return bytesWritten;
+  }
+
+  public void setBytesWritten(long bytesWritten) {
+    this.bytesWritten = bytesWritten;
+  }
+
+  public List<Pair<String, String>> getPartitionPathRecordKey() {
+    return partitionPathRecordKey;
+  }
+
+  public void setPartitionPathRecordKey(List<Pair<String, String>> partitionPathRecordKey) {
+    this.partitionPathRecordKey = partitionPathRecordKey;
+  }
+
+  public long getRecordsWritten() {
+    return recordsWritten;
+  }
+
+  public void setRecordsWritten(long recordsWritten) {
+    this.recordsWritten = recordsWritten;
+  }
+
+}
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterAdapter.java
similarity index 56%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterAdapter.java
index 7dde326..c941458 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterAdapter.java
@@ -16,28 +16,19 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
+package org.apache.hudi.integ.testsuite.writer;
 
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.spark.api.java.JavaRDD;
 
-import org.apache.avro.generic.GenericRecord;
+public interface DeltaWriterAdapter<I> {
 
-import java.io.Serializable;
+  List<DeltaWriteStats> write(Iterator<I> input) throws IOException;
 
-/**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
- */
-public abstract class KeyGenerator implements Serializable {
-
-  protected transient TypedProperties config;
+  interface SparkBasedDeltaWriter<J> {
 
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
+    JavaRDD<DeltaWriteStats> write(J input) throws IOException;
   }
-
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
 }
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterFactory.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterFactory.java
new file mode 100644
index 0000000..b4d9b9f
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DeltaWriterFactory.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.integ.testsuite.writer;
+
+import java.io.IOException;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig;
+import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
+
+/**
+ * A factory to help instantiate different {@link DeltaWriterAdapter}s depending on the {@link DeltaOutputMode} and
+ * {@link DeltaInputType}.
+ */
+public class DeltaWriterFactory {
+
+  private DeltaWriterFactory() {
+  }
+
+  public static DeltaWriterAdapter getDeltaWriterAdapter(DeltaConfig config, Integer batchId) throws IOException {
+    switch (config.getDeltaOutputMode()) {
+      case DFS:
+        switch (config.getDeltaInputType()) {
+          case AVRO:
+            DFSDeltaConfig dfsDeltaConfig = (DFSDeltaConfig) config;
+            dfsDeltaConfig.setBatchId(batchId);
+            DeltaInputWriter<GenericRecord> fileDeltaInputGenerator = new AvroFileDeltaInputWriter(
+                dfsDeltaConfig.getConfiguration(),
+                StringUtils
+                    .join(new String[]{dfsDeltaConfig.getDeltaBasePath(), dfsDeltaConfig.getBatchId().toString()},
+                        "/"), dfsDeltaConfig.getSchemaStr(), dfsDeltaConfig.getMaxFileSize());
+            return new DFSDeltaWriterAdapter(fileDeltaInputGenerator);
+          default:
+            throw new IllegalArgumentException("Invalid delta input format " + config.getDeltaInputType());
+        }
+      default:
+        throw new IllegalArgumentException("Invalid delta input type " + config.getDeltaOutputMode());
+    }
+  }
+}
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/SparkAvroDeltaInputWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/SparkAvroDeltaInputWriter.java
new file mode 100644
index 0000000..920f06e
--- /dev/null
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/SparkAvroDeltaInputWriter.java
@@ -0,0 +1,69 @@
+/*
+ * 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.hudi.integ.testsuite.writer;
+
+import java.io.IOException;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.SparkSession;
+
+/**
+ * Spark based avro delta input writer. We don't use this yet since we cannot control result file size.
+ */
+public class SparkAvroDeltaInputWriter implements DeltaInputWriter<JavaRDD<GenericRecord>> {
+
+  private static final String AVRO_FORMAT_PACKAGE = "avro";
+  public SparkSession sparkSession;
+  private String schemaStr;
+  // TODO : the base path has to be a new path every time for spark avro
+  private String basePath;
+
+  public SparkAvroDeltaInputWriter(SparkSession sparkSession, String schemaStr, String basePath) {
+    this.sparkSession = sparkSession;
+    this.schemaStr = schemaStr;
+    this.basePath = basePath;
+  }
+
+  @Override
+  public void writeData(JavaRDD<GenericRecord> iData) throws IOException {
+    AvroConversionUtils.createDataFrame(iData.rdd(), schemaStr, sparkSession).write()
+        .format(AVRO_FORMAT_PACKAGE).save(basePath);
+  }
+
+  @Override
+  public boolean canWrite() {
+    throw new UnsupportedOperationException("not applicable for spark based writer");
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+
+  @Override
+  public DeltaWriteStats getDeltaWriteStats() {
+    throw new UnsupportedOperationException("not applicable for spark based writer");
+  }
+
+  @Override
+  public DeltaInputWriter getNewWriter() throws IOException {
+    throw new UnsupportedOperationException("not applicable for spark based writer");
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/TestDFSHoodieTestSuiteWriterAdapter.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/TestDFSHoodieTestSuiteWriterAdapter.java
new file mode 100644
index 0000000..f6d9073
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/TestDFSHoodieTestSuiteWriterAdapter.java
@@ -0,0 +1,138 @@
+/*
+ * 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.hudi.integ.testsuite;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig;
+import org.apache.hudi.integ.testsuite.generator.FlexibleSchemaRecordGenerationIterator;
+import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
+import org.apache.hudi.integ.testsuite.utils.TestUtils;
+import org.apache.hudi.integ.testsuite.writer.AvroFileDeltaInputWriter;
+import org.apache.hudi.integ.testsuite.writer.DFSDeltaWriterAdapter;
+import org.apache.hudi.integ.testsuite.writer.DeltaInputWriter;
+import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
+import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats;
+import org.apache.hudi.integ.testsuite.writer.DeltaWriterAdapter;
+import org.apache.hudi.integ.testsuite.writer.DeltaWriterFactory;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+public class TestDFSHoodieTestSuiteWriterAdapter extends UtilitiesTestBase {
+
+  private FilebasedSchemaProvider schemaProvider;
+  private static final String COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/";
+
+  @BeforeAll
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+  }
+
+  @AfterAll
+  public static void cleanupClass() {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFSWithAbsoluteScope(
+        System.getProperty("user.dir") + "/.." + COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH,
+        "complex-source.avsc"), jsc);
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+  }
+
+  @Test
+  public void testDFSOneFileWrite() throws IOException {
+
+    DeltaInputWriter<GenericRecord> mockFileSinkWriter = Mockito.mock(AvroFileDeltaInputWriter.class);
+    DeltaWriteStats mockDeltaWriteStats = Mockito.mock(DeltaWriteStats.class);
+    when(mockFileSinkWriter.getNewWriter()).thenReturn(mockFileSinkWriter);
+    when(mockFileSinkWriter.canWrite()).thenReturn(true);
+    when(mockFileSinkWriter.getDeltaWriteStats()).thenReturn(mockDeltaWriteStats);
+
+    DeltaWriterAdapter<GenericRecord> dfsDeltaWriterAdapter = new DFSDeltaWriterAdapter(mockFileSinkWriter);
+
+    JavaRDD<GenericRecord> records = TestUtils.makeRDD(jsc, 10);
+
+    dfsDeltaWriterAdapter.write(records.collect().iterator());
+    Mockito.verify(mockFileSinkWriter, times(10)).canWrite();
+    Mockito.verify(mockFileSinkWriter, times(1)).close();
+  }
+
+  @Test
+  public void testDFSTwoFilesWriteWithRollover() throws IOException {
+
+    DeltaInputWriter<GenericRecord> mockFileSinkWriter = Mockito.mock(AvroFileDeltaInputWriter.class);
+    DeltaWriteStats mockDeltaWriteStats = Mockito.mock(DeltaWriteStats.class);
+    when(mockFileSinkWriter.getNewWriter()).thenReturn(mockFileSinkWriter);
+    when(mockFileSinkWriter.canWrite()).thenReturn(false, true);
+    when(mockFileSinkWriter.getDeltaWriteStats()).thenReturn(mockDeltaWriteStats);
+
+    DeltaWriterAdapter<GenericRecord> dfsDeltaWriterAdapter = new DFSDeltaWriterAdapter(mockFileSinkWriter);
+
+    Iterator<GenericRecord> mockIterator = Mockito.mock(Iterator.class);
+    when(mockIterator.hasNext()).thenReturn(true, true, true, false);
+
+    dfsDeltaWriterAdapter.write(mockIterator);
+    Mockito.verify(mockFileSinkWriter, times(2)).canWrite();
+    Mockito.verify(mockFileSinkWriter, times(1)).getNewWriter();
+    Mockito.verify(mockFileSinkWriter, times(2)).close();
+  }
+
+  @Test
+  public void testDFSWorkloadSinkWithMultipleFilesFunctional() throws IOException {
+    DeltaConfig dfsSinkConfig = new DFSDeltaConfig(DeltaOutputMode.DFS, DeltaInputType.AVRO,
+        new SerializableConfiguration(jsc.hadoopConfiguration()), dfsBasePath, dfsBasePath,
+        schemaProvider.getSourceSchema().toString(), 10240L);
+    DeltaWriterAdapter<GenericRecord> dfsDeltaWriterAdapter = DeltaWriterFactory
+        .getDeltaWriterAdapter(dfsSinkConfig, 1);
+    FlexibleSchemaRecordGenerationIterator itr = new FlexibleSchemaRecordGenerationIterator(1000,
+        schemaProvider.getSourceSchema().toString());
+    dfsDeltaWriterAdapter.write(itr);
+    FileSystem fs = FSUtils.getFs(dfsBasePath, jsc.hadoopConfiguration());
+    FileStatus[] fileStatuses = fs.listStatus(new Path(dfsBasePath));
+    // Since maxFileSize was 10240L and we produced 1K records each close to 1K size, we should produce more than
+    // 1 file
+    assertTrue(fileStatuses.length > 0);
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/TestFileDeltaInputWriter.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/TestFileDeltaInputWriter.java
new file mode 100644
index 0000000..f5dbd54
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/TestFileDeltaInputWriter.java
@@ -0,0 +1,147 @@
+/*
+ * 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.hudi.integ.testsuite;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.reader.SparkBasedReader;
+import org.apache.hudi.integ.testsuite.writer.AvroFileDeltaInputWriter;
+import org.apache.hudi.integ.testsuite.writer.DeltaInputWriter;
+import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats;
+import org.apache.hudi.integ.testsuite.generator.GenericRecordFullPayloadGenerator;
+import org.apache.hudi.integ.testsuite.reader.SparkBasedReader;
+import org.apache.hudi.integ.testsuite.writer.AvroFileDeltaInputWriter;
+import org.apache.hudi.integ.testsuite.writer.DeltaInputWriter;
+import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class TestFileDeltaInputWriter extends UtilitiesTestBase {
+
+  private FilebasedSchemaProvider schemaProvider;
+  private static final String COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/";
+
+  @BeforeAll
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+  }
+
+  @AfterAll
+  public static void cleanupClass() {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFSWithAbsoluteScope(System.getProperty("user.dir") + "/.."
+        + COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH, "complex-source.avsc"), jsc);
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+  }
+
+  @Test
+  public void testAvroFileSinkWriter() throws IOException {
+    // 1. Create a Avro File Sink Writer
+    DeltaInputWriter<GenericRecord> fileSinkWriter =
+        new AvroFileDeltaInputWriter(jsc.hadoopConfiguration(), dfsBasePath + "/input", schemaProvider.getSourceSchema()
+            .toString(), 1024 * 1024L);
+    GenericRecordFullPayloadGenerator payloadGenerator =
+        new GenericRecordFullPayloadGenerator(schemaProvider.getSourceSchema());
+    // 2. Generate 100 avro payloads and write them to an avro file
+    IntStream.range(0, 100).forEach(a -> {
+      try {
+        fileSinkWriter.writeData(payloadGenerator.getNewPayload());
+      } catch (IOException io) {
+        throw new UncheckedIOException(io);
+      }
+    });
+    fileSinkWriter.close();
+    DeltaWriteStats deltaWriteStats = fileSinkWriter.getDeltaWriteStats();
+    FileSystem fs = FSUtils.getFs(dfsBasePath, jsc.hadoopConfiguration());
+    FileStatus[] fileStatuses = fs.listStatus(new Path(deltaWriteStats.getFilePath()));
+    // Atleast 1 file was written
+    assertEquals(1, fileStatuses.length);
+    // File length should be greater than 0
+    assertTrue(fileStatuses[0].getLen() > 0);
+    // File length should be the same as the number of bytes written
+    assertTrue(deltaWriteStats.getBytesWritten() > 0);
+    List<String> paths = Arrays.asList(fs.globStatus(new Path(dfsBasePath + "/*/*.avro")))
+        .stream().map(f -> f.getPath().toString()).collect(Collectors.toList());
+    JavaRDD<GenericRecord> writtenRecords =
+        SparkBasedReader.readAvro(sparkSession, schemaProvider.getSourceSchema().toString(), paths, Option.empty(),
+            Option.empty());
+    // Number of records written should be 100
+    assertEquals(writtenRecords.count(), 100);
+    // Number of records in file should match with the stats
+    assertEquals(writtenRecords.count(), deltaWriteStats.getRecordsWritten());
+  }
+
+  @Test
+  public void testAvroFileSinkCreateNewWriter() throws IOException {
+    // 1. Create a Avro File Sink Writer
+    DeltaInputWriter<GenericRecord> fileSinkWriter =
+        new AvroFileDeltaInputWriter(jsc.hadoopConfiguration(), dfsBasePath,
+            schemaProvider.getSourceSchema().toString(),
+            1024 * 1024L);
+    GenericRecordFullPayloadGenerator payloadGenerator =
+        new GenericRecordFullPayloadGenerator(schemaProvider.getSourceSchema());
+    // 2. Generate 100 avro payloads and write them to an avro file
+    IntStream.range(0, 100).forEach(a -> {
+      try {
+        fileSinkWriter.writeData(payloadGenerator.getNewPayload());
+      } catch (IOException io) {
+        throw new UncheckedIOException(io);
+      }
+    });
+    fileSinkWriter.close();
+    String oldFilePath = fileSinkWriter.getDeltaWriteStats().getFilePath();
+    assertFalse(oldFilePath == null);
+    DeltaInputWriter<GenericRecord> newFileSinkWriter = fileSinkWriter.getNewWriter();
+    newFileSinkWriter.close();
+    DeltaWriteStats newStats = newFileSinkWriter.getDeltaWriteStats();
+    assertEquals(newStats.getBytesWritten(), 3674);
+    assertEquals(newStats.getRecordsWritten(), 0);
+    assertTrue(newStats.getFilePath() != null);
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/configuration/TestWorkflowBuilder.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/configuration/TestWorkflowBuilder.java
new file mode 100644
index 0000000..0ffe8a9
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/configuration/TestWorkflowBuilder.java
@@ -0,0 +1,80 @@
+/*
+ * 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.hudi.integ.testsuite.configuration;
+
+import static junit.framework.Assert.assertTrue;
+import static junit.framework.TestCase.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.WorkflowDag;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode;
+import org.junit.jupiter.api.Test;
+
+public class TestWorkflowBuilder {
+
+  @Test
+  public void testWorkloadOperationSequenceBuilder() {
+
+    DagNode root = new InsertNode(DeltaConfig.Config.newBuilder()
+        .withNumRecordsToInsert(10000)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(2)
+        .withRecordSize(1000).build());
+
+    DagNode child1 = new UpsertNode(DeltaConfig.Config.newBuilder()
+        .withNumRecordsToUpdate(10000)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(2)
+        .withRecordSize(1000).build());
+
+    root.addChildNode(child1);
+    child1.addParentNode(root);
+    List<DagNode> rootNodes = new ArrayList<>();
+    rootNodes.add(root);
+    WorkflowDag workflowDag = new WorkflowDag(rootNodes);
+
+    assertEquals(workflowDag.getNodeList().size(), 1);
+    assertEquals(((DagNode) workflowDag.getNodeList().get(0)).getChildNodes().size(), 1);
+    DagNode dagNode = (DagNode) workflowDag.getNodeList().get(0);
+    assertTrue(dagNode instanceof InsertNode);
+    DeltaConfig.Config config = dagNode.getConfig();
+    assertEquals(config.getNumInsertPartitions(), 1);
+    assertEquals(config.getRecordSize(), 1000);
+    assertEquals(config.getRepeatCount(), 2);
+    assertEquals(config.getNumRecordsInsert(), 10000);
+    assertEquals(config.getNumRecordsUpsert(), 0);
+    dagNode = (DagNode) ((DagNode) workflowDag.getNodeList().get(0)).getChildNodes().get(0);
+    assertTrue(dagNode instanceof UpsertNode);
+    config = dagNode.getConfig();
+    assertEquals(config.getNumInsertPartitions(), 1);
+    assertEquals(config.getRecordSize(), 1000);
+    assertEquals(config.getRepeatCount(), 2);
+    assertEquals(config.getNumRecordsInsert(), 0);
+    assertEquals(config.getNumRecordsUpsert(), 10000);
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/converter/TestUpdateConverter.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/converter/TestUpdateConverter.java
new file mode 100644
index 0000000..98e0991
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/converter/TestUpdateConverter.java
@@ -0,0 +1,93 @@
+/*
+ * 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.hudi.integ.testsuite.converter;
+
+import static junit.framework.TestCase.assertTrue;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.integ.testsuite.utils.TestUtils;
+import org.apache.hudi.integ.testsuite.utils.TestUtils;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import scala.Tuple2;
+
+public class TestUpdateConverter {
+
+  private JavaSparkContext jsc;
+
+  @BeforeEach
+  public void setup() throws Exception {
+    jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[1]");
+
+  }
+
+  @AfterEach
+  public void teardown() {
+    jsc.stop();
+  }
+
+  @Test
+  public void testGenerateUpdateRecordsFromInputRecords() throws Exception {
+    JavaRDD<GenericRecord> inputRDD = TestUtils.makeRDD(jsc, 10);
+    String schemaStr = inputRDD.take(1).get(0).getSchema().toString();
+    int minPayloadSize = 1000;
+    // 2. DFS converter reads existing records and generates random updates for the same row keys
+    UpdateConverter updateConverter = new UpdateConverter(schemaStr, minPayloadSize,
+        Arrays.asList("timestamp"), Arrays.asList("_row_key"));
+    List<String> insertRowKeys = inputRDD.map(r -> r.get("_row_key").toString()).collect();
+    assertTrue(inputRDD.count() == 10);
+    JavaRDD<GenericRecord> outputRDD = updateConverter.convert(inputRDD);
+    List<String> updateRowKeys = outputRDD.map(row -> row.get("_row_key").toString()).collect();
+    // The insert row keys should be the same as update row keys
+    assertTrue(insertRowKeys.containsAll(updateRowKeys));
+    Map<String, GenericRecord> inputRecords = inputRDD.mapToPair(r -> new Tuple2<>(r.get("_row_key").toString(), r))
+        .collectAsMap();
+    List<GenericRecord> updateRecords = outputRDD.collect();
+    updateRecords.stream().forEach(updateRecord -> {
+      GenericRecord inputRecord = inputRecords.get(updateRecord.get("_row_key").toString());
+      assertTrue(areRecordsDifferent(inputRecord, updateRecord));
+    });
+
+  }
+
+  /**
+   * Checks if even a single field in the 2 records is different (except the row key which is the same for an update).
+   */
+  private boolean areRecordsDifferent(GenericRecord in, GenericRecord up) {
+    for (Field field : in.getSchema().getFields()) {
+      if (field.name() == "_row_key") {
+        continue;
+      } else {
+        // Just convert all types to string for now since all are primitive
+        if (in.get(field.name()).toString() != up.get(field.name()).toString()) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/ComplexDagGenerator.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/ComplexDagGenerator.java
new file mode 100644
index 0000000..653a9e4
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/ComplexDagGenerator.java
@@ -0,0 +1,81 @@
+/*
+ * 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.hudi.integ.testsuite.dag;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.ValidateNode;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.ValidateNode;
+import org.apache.spark.api.java.JavaRDD;
+
+public class ComplexDagGenerator implements WorkflowDagGenerator {
+
+  @Override
+  public WorkflowDag build() {
+    DagNode root = new InsertNode(Config.newBuilder()
+        .withNumRecordsToInsert(1000)
+        .withNumInsertPartitions(3)
+        .withRecordSize(1000).build());
+
+    DagNode child1 = new UpsertNode(Config.newBuilder()
+        .withNumRecordsToUpdate(999)
+        .withNumRecordsToInsert(1000)
+        .withNumUpsertFiles(1)
+        .withNumUpsertPartitions(1)
+        .withNumInsertPartitions(1)
+        .withRecordSize(10000).build());
+
+    Function<List<DagNode<JavaRDD<WriteStatus>>>, Boolean> function = (dagNodes) -> {
+      DagNode<JavaRDD<WriteStatus>> parent1 = dagNodes.get(0);
+      List<WriteStatus> statuses = parent1.getResult().collect();
+      long totalRecordsTouched = statuses.stream().map(st -> st.getStat().getNumUpdateWrites() + st.getStat()
+          .getNumInserts()).reduce((a, b) -> a + b).get();
+      boolean b1 = totalRecordsTouched == parent1.getConfig().getNumRecordsInsert()
+          + parent1.getConfig().getNumRecordsUpsert();
+      boolean b2 = statuses.size() > parent1.getConfig().getNumUpsertFiles();
+
+      DagNode<JavaRDD<WriteStatus>> parent2 = parent1.getParentNodes().get(0);
+      statuses = parent2.getResult().collect();
+      totalRecordsTouched = statuses.stream().map(st -> st.getStat().getNumUpdateWrites() + st.getStat()
+          .getNumInserts()).reduce((a, b) -> a + b).get();
+      boolean b3 = totalRecordsTouched == parent2.getConfig().getNumRecordsInsert()
+          * parent2.getConfig().getNumInsertPartitions() + parent2.getConfig().getNumRecordsUpsert();
+      return b1 & b2 & b3;
+    };
+    DagNode child2 = new ValidateNode(Config.newBuilder().build(), function);
+
+    root.addChildNode(child1);
+    // child1.addParentNode(root);
+    child1.addChildNode(child2);
+    // child2.addParentNode(child1);
+    List<DagNode> rootNodes = new ArrayList<>();
+    rootNodes.add(root);
+    return new WorkflowDag(rootNodes);
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/HiveSyncDagGenerator.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/HiveSyncDagGenerator.java
new file mode 100644
index 0000000..79eb246
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/HiveSyncDagGenerator.java
@@ -0,0 +1,58 @@
+/*
+ * 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.hudi.integ.testsuite.dag;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.HiveSyncNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.HiveQueryNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.HiveSyncNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
+
+public class HiveSyncDagGenerator implements WorkflowDagGenerator {
+
+  @Override
+  public WorkflowDag build() {
+    DagNode root = new InsertNode(Config.newBuilder()
+        .withNumRecordsToInsert(100)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(1)
+        .withRecordSize(1000).build());
+
+    DagNode child1 = new HiveSyncNode(Config.newBuilder().withHiveLocal(true).build());
+
+    root.addChildNode(child1);
+
+    DagNode child2 = new HiveQueryNode(Config.newBuilder().withHiveLocal(true).withHiveQueryAndResults(Arrays
+        .asList(Pair.of("select " + "count(*) from testdb1.table1 group " + "by rider having count(*) < 1", 0)))
+        .build());
+    child1.addChildNode(child2);
+
+    List<DagNode> rootNodes = new ArrayList<>();
+    rootNodes.add(root);
+    return new WorkflowDag(rootNodes);
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/HiveSyncDagGeneratorMOR.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/HiveSyncDagGeneratorMOR.java
new file mode 100644
index 0000000..1f9f7a4
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/HiveSyncDagGeneratorMOR.java
@@ -0,0 +1,58 @@
+/*
+ * 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.hudi.integ.testsuite.dag;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.HiveSyncNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.HiveQueryNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.HiveSyncNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
+
+public class HiveSyncDagGeneratorMOR implements WorkflowDagGenerator {
+
+  @Override
+  public WorkflowDag build() {
+    DagNode root = new InsertNode(Config.newBuilder()
+        .withNumRecordsToInsert(100)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(1)
+        .withRecordSize(1000).build());
+
+    DagNode child1 = new HiveSyncNode(Config.newBuilder().withHiveLocal(true).build());
+
+    root.addChildNode(child1);
+
+    DagNode child2 = new HiveQueryNode(Config.newBuilder().withHiveLocal(true).withHiveQueryAndResults(Arrays
+        .asList(Pair.of("select " + "count(*) from testdb1.table1_rt group " + "by rider having count(*) < 1", 0)))
+        .build());
+    child1.addChildNode(child2);
+
+    List<DagNode> rootNodes = new ArrayList<>();
+    rootNodes.add(root);
+    return new WorkflowDag(rootNodes);
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/TestDagUtils.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/TestDagUtils.java
new file mode 100644
index 0000000..b5a0cd3
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/dag/TestDagUtils.java
@@ -0,0 +1,95 @@
+/*
+ * 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.hudi.integ.testsuite.dag;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
+import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+public class TestDagUtils {
+
+  private static final String COW_DAG_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/complex-dag-cow.yaml";
+
+  @Test
+  public void testConvertDagToYaml() throws Exception {
+    ComplexDagGenerator dag = new ComplexDagGenerator();
+    String yaml = DagUtils.convertDagToYaml(dag.build());
+    System.out.println(yaml);
+  }
+
+  @Test
+  public void testConvertYamlToDag() throws Exception {
+    WorkflowDag dag = DagUtils.convertYamlToDag(UtilitiesTestBase.Helpers
+        .readFileFromAbsolutePath((System.getProperty("user.dir") + "/.." + COW_DAG_DOCKER_DEMO_RELATIVE_PATH)));
+    assertEquals(dag.getNodeList().size(), 1);
+    Assertions.assertEquals(((DagNode) dag.getNodeList().get(0)).getParentNodes().size(), 0);
+    assertEquals(((DagNode) dag.getNodeList().get(0)).getChildNodes().size(), 1);
+    DagNode firstChild = (DagNode) ((DagNode) dag.getNodeList().get(0)).getChildNodes().get(0);
+    assertEquals(firstChild.getParentNodes().size(), 1);
+    assertEquals(firstChild.getChildNodes().size(), 1);
+    assertEquals(((DagNode) firstChild.getChildNodes().get(0)).getChildNodes().size(), 1);
+  }
+
+  public static class ComplexDagGenerator implements WorkflowDagGenerator {
+
+    @Override
+    public WorkflowDag build() {
+      DagNode root = new InsertNode(Config.newBuilder()
+          .withNumRecordsToInsert(1000000)
+          .withNumInsertPartitions(1)
+          .withNumTimesToRepeat(2)
+          .withRecordSize(1000).build());
+
+      DagNode child1 = new InsertNode(Config.newBuilder()
+          .withNumRecordsToInsert(1000000)
+          .withNumInsertPartitions(1)
+          .withNumTimesToRepeat(2)
+          .withRecordSize(1000).build());
+
+      DagNode child2 = new InsertNode(Config.newBuilder()
+          .withNumRecordsToInsert(1000000)
+          .withNumInsertPartitions(1)
+          .withNumTimesToRepeat(2)
+          .withRecordSize(1000).build());
+
+      root.addChildNode(child1);
+      root.addChildNode(child2);
+
+      DagNode child3 = new InsertNode(Config.newBuilder()
+          .withNumRecordsToInsert(1000000)
+          .withNumInsertPartitions(1)
+          .withNumTimesToRepeat(2)
+          .withRecordSize(1000).build());
+
+      child2.addChildNode(child3);
+      List<DagNode> rootNodes = new ArrayList<>();
+      rootNodes.add(root);
+
+      return new WorkflowDag(rootNodes);
+    }
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/generator/TestGenericRecordPayloadEstimator.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/generator/TestGenericRecordPayloadEstimator.java
new file mode 100644
index 0000000..85d5358
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/generator/TestGenericRecordPayloadEstimator.java
@@ -0,0 +1,56 @@
+/*
+ * 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.hudi.integ.testsuite.generator;
+
+import static junit.framework.TestCase.assertEquals;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+import org.junit.jupiter.api.Test;
+
+public class TestGenericRecordPayloadEstimator {
+
+  private static final String SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/source.avsc";
+  private static final String COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH =
+      "/docker/demo/config/test-suite/complex-source.avsc";
+
+  @Test
+  public void testSimpleSchemaSize() throws Exception {
+    Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
+        .readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." + SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
+    GenericRecordFullPayloadSizeEstimator estimator =
+        new GenericRecordFullPayloadSizeEstimator(schema);
+    Pair<Integer, Integer> estimateAndNumComplexFields = estimator.typeEstimateAndNumComplexFields();
+    assertEquals(estimateAndNumComplexFields.getRight().intValue(), 0);
+    assertEquals(estimateAndNumComplexFields.getLeft().intValue(), 156);
+  }
+
+  @Test
+  public void testComplexSchemaSize() throws Exception {
+    Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers.readFileFromAbsolutePath(
+        System.getProperty("user.dir") + "/.." + COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
+    GenericRecordFullPayloadSizeEstimator estimator =
+        new GenericRecordFullPayloadSizeEstimator(schema);
+    Pair<Integer, Integer> estimateAndNumComplexFields = estimator.typeEstimateAndNumComplexFields();
+    assertEquals(estimateAndNumComplexFields.getRight().intValue(), 1);
+    assertEquals(estimateAndNumComplexFields.getLeft().intValue(), 1278);
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/generator/TestGenericRecordPayloadGenerator.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/generator/TestGenericRecordPayloadGenerator.java
new file mode 100644
index 0000000..886fb16
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/generator/TestGenericRecordPayloadGenerator.java
@@ -0,0 +1,130 @@
+/*
+ * 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.hudi.integ.testsuite.generator;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.IntStream;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+import org.junit.jupiter.api.Test;
+
+public class TestGenericRecordPayloadGenerator {
+
+  private static final String SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/source.avsc";
+  private static final String COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH =
+      "/docker/demo/config/test-suite/complex-source.avsc";
+
+  @Test
+  public void testSimplePayload() throws Exception {
+    Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
+        .readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." + SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
+    GenericRecordFullPayloadGenerator payloadGenerator = new GenericRecordFullPayloadGenerator(schema);
+    GenericRecord record = payloadGenerator.getNewPayload();
+    // The generated payload should validate with the provided schema
+    payloadGenerator.validate(record);
+  }
+
+  @Test
+  public void testComplexPayload() throws IOException {
+    Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
+        .readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." +
+            COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
+    GenericRecordFullPayloadGenerator payloadGenerator = new GenericRecordFullPayloadGenerator(schema);
+    GenericRecord record = payloadGenerator.getNewPayload();
+    // The generated payload should validate with the provided schema
+    assertTrue(payloadGenerator.validate(record));
+  }
+
+  @Test
+  public void testComplexPartialPayload() throws IOException {
+    Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
+        .readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." +
+            COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
+    GenericRecordPartialPayloadGenerator payloadGenerator = new GenericRecordPartialPayloadGenerator(schema);
+    IntStream.range(0, 10).forEach(a -> {
+      GenericRecord record = payloadGenerator.getNewPayload();
+      // The generated payload should validate with the provided schema
+      assertTrue(payloadGenerator.validate(record));
+    });
+  }
+
+  @Test
+  public void testUpdatePayloadGenerator() throws IOException {
+    Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
+        .readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." + SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
+    GenericRecordFullPayloadGenerator payloadGenerator = new GenericRecordFullPayloadGenerator(schema);
+    List<String> insertRowKeys = new ArrayList<>();
+    List<String> updateRowKeys = new ArrayList<>();
+    List<Long> insertTimeStamps = new ArrayList<>();
+    List<Long> updateTimeStamps = new ArrayList<>();
+    List<GenericRecord> records = new ArrayList<>();
+    // Generate 10 new records
+    IntStream.range(0, 10).forEach(a -> {
+      GenericRecord record = payloadGenerator.getNewPayload();
+      records.add(record);
+      insertRowKeys.add(record.get("_row_key").toString());
+      insertTimeStamps.add((Long) record.get("timestamp"));
+    });
+    List<String> blacklistFields = Arrays.asList("_row_key");
+    records.stream().forEach(a -> {
+      // Generate 10 updated records
+      GenericRecord record = payloadGenerator.getUpdatePayload(a, blacklistFields);
+      updateRowKeys.add(record.get("_row_key").toString());
+      updateTimeStamps.add((Long) record.get("timestamp"));
+    });
+    // The row keys from insert payloads should match all the row keys from the update payloads
+    assertTrue(insertRowKeys.containsAll(updateRowKeys));
+    // The timestamp field for the insert payloads should not all match with the update payloads
+    assertFalse(insertTimeStamps.containsAll(updateTimeStamps));
+  }
+
+  @Test
+  public void testSimplePayloadWithLargeMinSize() throws Exception {
+    Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
+        .readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." + SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
+    int minPayloadSize = 1000;
+    GenericRecordFullPayloadGenerator payloadGenerator = new GenericRecordFullPayloadGenerator(schema,
+        minPayloadSize);
+    GenericRecord record = payloadGenerator.getNewPayload();
+    // The payload generated is less than minPayloadSize due to no collections present
+    assertTrue(HoodieAvroUtils.avroToBytes(record).length < minPayloadSize);
+  }
+
+  @Test
+  public void testComplexPayloadWithLargeMinSize() throws Exception {
+    Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
+        .readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." +
+            COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
+    int minPayloadSize = 10000;
+    GenericRecordFullPayloadGenerator payloadGenerator = new GenericRecordFullPayloadGenerator(
+        schema, minPayloadSize);
+    GenericRecord record = payloadGenerator.getNewPayload();
+    // The payload generated should be within 10% extra of the minPayloadSize
+    assertTrue(HoodieAvroUtils.avroToBytes(record).length < minPayloadSize + 0.1 * minPayloadSize);
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java
new file mode 100644
index 0000000..9d0d104
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java
@@ -0,0 +1,224 @@
+/*
+ * 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.hudi.integ.testsuite.job;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.UUID;
+import java.util.stream.Stream;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob;
+import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig;
+import org.apache.hudi.integ.testsuite.dag.ComplexDagGenerator;
+import org.apache.hudi.integ.testsuite.dag.HiveSyncDagGenerator;
+import org.apache.hudi.integ.testsuite.dag.HiveSyncDagGeneratorMOR;
+import org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator;
+import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
+import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
+import org.apache.hudi.keygen.TimestampBasedKeyGenerator;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.hudi.utilities.sources.AvroDFSSource;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+public class TestHoodieTestSuiteJob extends UtilitiesTestBase {
+
+  private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with useDeltaStreamer={0}, tableType={1}";
+  private static final String BASE_PROPERTIES_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/base"
+      + ".properties";
+  private static final String SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/source.avsc";
+  private static final String TARGET_SCHEMA_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/target.avsc";
+  private static final String COW_DAG_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/complex-dag-cow.yaml";
+  private static final String MOR_DAG_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/complex-dag-mor.yaml";
+
+  public static Stream<Arguments> configParams() {
+    Object[][] data =
+        new Object[][] {{false, "COPY_ON_WRITE"}};
+    return Stream.of(data).map(Arguments::of);
+  }
+
+  @BeforeAll
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+    // prepare the configs.
+    UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.."
+        + BASE_PROPERTIES_DOCKER_DEMO_RELATIVE_PATH, dfs, dfsBasePath + "/base.properties");
+    UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.."
+        + SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH, dfs, dfsBasePath + "/source.avsc");
+    UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.."
+        + TARGET_SCHEMA_DOCKER_DEMO_RELATIVE_PATH, dfs, dfsBasePath + "/target.avsc");
+
+    UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.."
+        + COW_DAG_DOCKER_DEMO_RELATIVE_PATH, dfs, dfsBasePath + "/complex-dag-cow.yaml");
+    UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.."
+        + MOR_DAG_DOCKER_DEMO_RELATIVE_PATH, dfs, dfsBasePath + "/complex-dag-mor.yaml");
+
+    TypedProperties props = new TypedProperties();
+    props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
+    props.setProperty("hoodie.datasource.write.partitionpath.field", "timestamp");
+    props.setProperty("hoodie.deltastreamer.keygen.timebased.timestamp.type", "UNIX_TIMESTAMP");
+    props.setProperty("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyy/MM/dd");
+    props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
+    props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/source.avsc");
+    props.setProperty("hoodie.deltastreamer.source.dfs.root", dfsBasePath + "/input");
+    props.setProperty("hoodie.datasource.hive_sync.assume_date_partitioning", "true");
+    props.setProperty("hoodie.datasource.write.keytranslator.class", "org.apache.hudi"
+        + ".DayBasedPartitionPathKeyTranslator");
+    props.setProperty("hoodie.compact.inline.max.delta.commits", "3");
+    props.setProperty("hoodie.parquet.max.file.size", "1024000");
+    props.setProperty("hoodie.compact.inline.max.delta.commits", "0");
+    // Hive Configs
+    props.setProperty(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), "jdbc:hive2://127.0.0.1:9999/");
+    props.setProperty(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), "testdb1");
+    props.setProperty(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), "table1");
+    props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "datestr");
+    props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), TimestampBasedKeyGenerator.class.getName());
+    UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/test-source"
+        + ".properties");
+
+    // Properties used for the delta-streamer which incrementally pulls from upstream DFS Avro source and
+    // writes to downstream hudi table
+    TypedProperties downstreamProps = new TypedProperties();
+    downstreamProps.setProperty("include", "base.properties");
+    downstreamProps.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
+    downstreamProps.setProperty("hoodie.datasource.write.partitionpath.field", "timestamp");
+
+    // Source schema is the target schema of upstream table
+    downstreamProps.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
+    downstreamProps.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/source.avsc");
+    UtilitiesTestBase.Helpers.savePropsToDFS(downstreamProps, dfs,
+        dfsBasePath + "/test-downstream-source.properties");
+    // these tests cause a lot of log verbosity from spark, turning it down
+    Logger.getLogger("org.apache.spark").setLevel(Level.WARN);
+  }
+
+  @AfterAll
+  public static void cleanupClass() {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+  }
+
+  // Tests in this class add to the test build time significantly. Since this is a Integration Test (end to end), we
+  // would like to run this as a nightly build which is a TODO.
+  // TODO : Clean up input / result paths after each test
+  @MethodSource("configParams")
+  public void testDagWithInsertUpsertAndValidate(boolean useDeltaStreamer, String tableType) throws Exception {
+    dfs.delete(new Path(dfsBasePath + "/input"), true);
+    dfs.delete(new Path(dfsBasePath + "/result"), true);
+    String inputBasePath = dfsBasePath + "/input/" + UUID.randomUUID().toString();
+    String outputBasePath = dfsBasePath + "/result/" + UUID.randomUUID().toString();
+    HoodieTestSuiteConfig cfg = makeConfig(inputBasePath, outputBasePath, useDeltaStreamer, tableType);
+    cfg.workloadDagGenerator = ComplexDagGenerator.class.getName();
+    HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc);
+    hoodieTestSuiteJob.runTestSuite();
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(new Configuration(), cfg.targetBasePath);
+    assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 2);
+  }
+
+  @MethodSource("configParams")
+  public void testHiveSync(boolean useDeltaStreamer, String tableType) throws Exception {
+    dfs.delete(new Path(dfsBasePath + "/input"), true);
+    dfs.delete(new Path(dfsBasePath + "/result"), true);
+    String inputBasePath = dfsBasePath + "/input";
+    String outputBasePath = dfsBasePath + "/result";
+    HoodieTestSuiteConfig cfg = makeConfig(inputBasePath, outputBasePath, useDeltaStreamer, tableType);
+    if (tableType == HoodieTableType.COPY_ON_WRITE.name()) {
+      cfg.workloadDagGenerator = HiveSyncDagGenerator.class.getName();
+    } else {
+      cfg.workloadDagGenerator = HiveSyncDagGeneratorMOR.class.getName();
+    }
+    HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc);
+    hoodieTestSuiteJob.runTestSuite();
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(new Configuration(), cfg.targetBasePath);
+    assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 1);
+  }
+
+  @MethodSource("configParams")
+  public void testCOWFullDagFromYaml(boolean useDeltaStreamer, String tableType) throws Exception {
+    dfs.delete(new Path(dfsBasePath + "/input"), true);
+    dfs.delete(new Path(dfsBasePath + "/result"), true);
+    String inputBasePath = dfsBasePath + "/input";
+    String outputBasePath = dfsBasePath + "/result";
+    HoodieTestSuiteConfig cfg = makeConfig(inputBasePath, outputBasePath, useDeltaStreamer, HoodieTableType
+        .COPY_ON_WRITE.name());
+    cfg.workloadYamlPath = dfsBasePath + "/complex-dag-cow.yaml";
+    HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc);
+    hoodieTestSuiteJob.runTestSuite();
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(new Configuration(), cfg.targetBasePath);
+    assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 5);
+  }
+
+  @MethodSource("configParams")
+  public void testMORFullDagFromYaml(boolean useDeltaStreamer, String tableType) throws Exception {
+    dfs.delete(new Path(dfsBasePath + "/input"), true);
+    dfs.delete(new Path(dfsBasePath + "/result"), true);
+    String inputBasePath = dfsBasePath + "/input";
+    String outputBasePath = dfsBasePath + "/result";
+    HoodieTestSuiteConfig cfg = makeConfig(inputBasePath, outputBasePath, useDeltaStreamer, HoodieTableType
+        .MERGE_ON_READ.name());
+    cfg.workloadYamlPath = dfsBasePath + "/complex-dag-mor.yaml";
+    HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc);
+    hoodieTestSuiteJob.runTestSuite();
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(new Configuration(), cfg.targetBasePath);
+    assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 7);
+  }
+
+  protected HoodieTestSuiteConfig makeConfig(String inputBasePath, String outputBasePath, boolean useDeltaStream,
+      String tableType) {
+    HoodieTestSuiteConfig cfg = new HoodieTestSuiteConfig();
+    cfg.targetBasePath = outputBasePath;
+    cfg.inputBasePath = inputBasePath;
+    cfg.targetTableName = "table1";
+    cfg.tableType = tableType;
+    cfg.sourceClassName = AvroDFSSource.class.getName();
+    cfg.sourceOrderingField = "timestamp";
+    cfg.propsFilePath = dfsBasePath + "/test-source.properties";
+    cfg.outputTypeName = DeltaOutputMode.DFS.name();
+    cfg.inputFormatName = DeltaInputType.AVRO.name();
+    cfg.limitFileSize = 1024 * 1024L;
+    cfg.sourceLimit = 20000000;
+    cfg.workloadDagGenerator = WorkflowDagGenerator.class.getName();
+    cfg.schemaProviderClassName = FilebasedSchemaProvider.class.getName();
+    cfg.useDeltaStreamer = useDeltaStream;
+    return cfg;
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSAvroDeltaInputReader.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSAvroDeltaInputReader.java
new file mode 100644
index 0000000..498cc62
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSAvroDeltaInputReader.java
@@ -0,0 +1,74 @@
+/*
+ * 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.hudi.integ.testsuite.reader;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.integ.testsuite.utils.TestUtils;
+import org.apache.hudi.integ.testsuite.utils.TestUtils;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+public class TestDFSAvroDeltaInputReader extends UtilitiesTestBase {
+
+  @BeforeAll
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+  }
+
+  @AfterAll
+  public static void cleanupClass() {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+  }
+
+  @Test
+  @Disabled
+  public void testDFSSinkReader() throws IOException {
+    FileSystem fs = FSUtils.getFs(dfsBasePath, new Configuration());
+    // Create 10 avro files with 10 records each
+    TestUtils.createAvroFiles(jsc, sparkSession, dfsBasePath, 10, 10);
+    FileStatus[] statuses = fs.globStatus(new Path(dfsBasePath + "/*/*.avro"));
+    DFSAvroDeltaInputReader reader =
+        new DFSAvroDeltaInputReader(sparkSession, TestUtils.getSchema().toString(), dfsBasePath, Option.empty(),
+            Option.empty());
+    assertEquals(reader.analyzeSingleFile(statuses[0].getPath().toString()), 5);
+    assertEquals(reader.read(100).count(), 100);
+    assertEquals(reader.read(1000).count(), 100);
+    assertEquals(reader.read(10).count(), 10);
+    assertTrue(reader.read(11).count() > 11);
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSHoodieDatasetInputReader.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSHoodieDatasetInputReader.java
new file mode 100644
index 0000000..89e586e
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSHoodieDatasetInputReader.java
@@ -0,0 +1,119 @@
+/*
+ * 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.hudi.integ.testsuite.reader;
+
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertTrue;
+
+import java.util.HashSet;
+import java.util.List;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.testutils.HoodieTestUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class TestDFSHoodieDatasetInputReader extends UtilitiesTestBase {
+
+  @BeforeAll
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+  }
+
+  @AfterAll
+  public static void cleanupClass() {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+    HoodieTestUtils.init(jsc.hadoopConfiguration(), dfsBasePath);
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+  }
+
+  @Test
+  public void testSimpleHoodieDatasetReader() throws Exception {
+
+    HoodieWriteConfig config = makeHoodieClientConfig();
+    HoodieWriteClient client = new HoodieWriteClient(jsc, config);
+    String commitTime = client.startCommit();
+    HoodieTestDataGenerator generator = new HoodieTestDataGenerator();
+    // Insert 100 records across 3 partitions
+    List<HoodieRecord> inserts = generator.generateInserts(commitTime, 100);
+    JavaRDD<WriteStatus> writeStatuses = client.upsert(jsc.parallelize(inserts), commitTime);
+    writeStatuses.count();
+
+    DFSHoodieDatasetInputReader reader = new DFSHoodieDatasetInputReader(jsc, config.getBasePath(),
+        HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())).toString());
+    // Try to read 100 records for the same partition path and same file ID
+    JavaRDD<GenericRecord> records = reader.read(1, 1, 100L);
+    assertTrue(records.count() <= 100);
+    assertEquals(new HashSet<>(records.map(p -> p.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).collect()).size(),
+        1);
+    assertEquals(new HashSet<>(records.map(p -> p.get(HoodieRecord.FILENAME_METADATA_FIELD)).collect()).size(),
+        1);
+
+    // Try to read 100 records for 3 partition paths and 3 different file ids
+    records = reader.read(3, 3, 100L);
+    assertTrue(records.count() <= 100);
+    assertEquals(new HashSet<>(records.map(p -> p.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).collect()).size(),
+        3);
+    assertEquals(new HashSet<>(records.map(p -> p.get(HoodieRecord.FILENAME_METADATA_FIELD)).collect()).size(),
+        3);
+
+    // Try to read 100 records for 3 partition paths and 50% records from each file
+    records = reader.read(3, 3, 0.5);
+    assertTrue(records.count() <= 100);
+    assertEquals(new HashSet<>(records.map(p -> p.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).collect()).size(),
+        3);
+    assertEquals(new HashSet<>(records.map(p -> p.get(HoodieRecord.FILENAME_METADATA_FIELD)).collect()).size(),
+        3);
+  }
+
+  private HoodieWriteConfig makeHoodieClientConfig() throws Exception {
+    return makeHoodieClientConfigBuilder().build();
+  }
+
+  private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception {
+    // Prepare the AvroParquetIO
+    return HoodieWriteConfig.newBuilder().withPath(dfsBasePath)
+        .withParallelism(2, 2)
+        .withSchema(HoodieTestDataGenerator
+            .TRIP_EXAMPLE_SCHEMA);
+  }
+
+}
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/utils/TestUtils.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/utils/TestUtils.java
new file mode 100644
index 0000000..4699324
--- /dev/null
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/utils/TestUtils.java
@@ -0,0 +1,63 @@
+/*
+ * 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.hudi.integ.testsuite.utils;
+
+import java.util.List;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.utilities.schema.RowBasedSchemaProvider;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+
+public class TestUtils {
+
+  /**
+   * Create a RDD of generic records for testing purposes.
+   */
+  public static JavaRDD<GenericRecord> makeRDD(JavaSparkContext jsc, int numRecords) {
+    return jsc.parallelize(generateGenericRecords(numRecords));
+  }
+
+  /**
+   * Generate generic records.
+   */
+  public static List<GenericRecord> generateGenericRecords(int numRecords) {
+    HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
+    return dataGenerator.generateGenericRecords(numRecords);
+  }
+
+  public static void createAvroFiles(JavaSparkContext jsc, SparkSession sparkSession, String basePath, int numFiles,
+      int numRecordsPerFile) {
+    Schema schema = HoodieTestDataGenerator.AVRO_SCHEMA;
+    for (int i = 0; i < numFiles; i++) {
+      JavaRDD<GenericRecord> rdd = makeRDD(jsc, numRecordsPerFile);
+      AvroConversionUtils.createDataFrame(rdd.rdd(), schema.toString(), sparkSession).write()
+          .format("avro").option("recordName", RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME)
+          .option("recordNamespace", RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE).save(basePath + "/" + i);
+    }
+  }
+
+  public static Schema getSchema() {
+    return HoodieTestDataGenerator.AVRO_SCHEMA;
+  }
+
+}
diff --git a/hudi-integ-test/src/test/resources/log4j-surefire-quiet.properties b/hudi-integ-test/src/test/resources/log4j-surefire-quiet.properties
index b21b5d4..61fbf78 100644
--- a/hudi-integ-test/src/test/resources/log4j-surefire-quiet.properties
+++ b/hudi-integ-test/src/test/resources/log4j-surefire-quiet.properties
@@ -15,8 +15,9 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 ###
-log4j.rootLogger=WARN, CONSOLE
-log4j.logger.org.apache.hudi=DEBUG
+log4j.rootLogger=ERROR, CONSOLE
+log4j.logger.org.apache.hudi=ERROR
+log4j.category.org.apache.spark=ERROR
 
 # CONSOLE is set to be a ConsoleAppender.
 log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
@@ -25,5 +26,5 @@ log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
 log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
 log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
 log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
-log4j.appender.CONSOLE.filter.a.LevelMin=WARN
+log4j.appender.CONSOLE.filter.a.LevelMin=ERROR
 log4j.appender.CONSOLE.filter.a.LevelMax=FATAL
diff --git a/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java b/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java
index 064e37c..513bb59 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java
@@ -95,7 +95,7 @@ public class QuickstartUtils {
     }
 
     public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName,
-                                                      double timestamp) {
+        double timestamp) {
       GenericRecord rec = new GenericData.Record(avroSchema);
       rec.put("uuid", rowKey);
       rec.put("ts", timestamp);
@@ -221,4 +221,4 @@ public class QuickstartUtils {
     demoConfigs.put("hoodie.upsert.shuffle.parallelism", "2");
     return demoConfigs;
   }
-}
+}
\ No newline at end of file
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java
index b3ab3d0..b6431fe 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java
@@ -37,6 +37,7 @@ public class ComplexKeyGenerator extends KeyGenerator {
 
   private static final String DEFAULT_PARTITION_PATH = "default";
   private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
+  public static final String DEFAULT_RECORD_KEY_SEPARATOR = ":";
 
   protected static final String NULL_RECORDKEY_PLACEHOLDER = "__null__";
   protected static final String EMPTY_RECORDKEY_PLACEHOLDER = "__empty__";
@@ -98,8 +99,16 @@ public class ComplexKeyGenerator extends KeyGenerator {
     recordKey.deleteCharAt(recordKey.length() - 1);
     if (keyIsNullEmpty) {
       throw new HoodieKeyException("recordKey values: \"" + recordKey + "\" for fields: "
-        + recordKeyFields.toString() + " cannot be entirely null or empty.");
+          + recordKeyFields.toString() + " cannot be entirely null or empty.");
     }
     return recordKey.toString();
   }
-}
+
+  public List<String> getRecordKeyFields() {
+    return recordKeyFields;
+  }
+
+  public List<String> getPartitionPathFields() {
+    return partitionPathFields;
+  }
+}
\ No newline at end of file
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
index 7dde326..b4d609d 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
@@ -40,4 +40,4 @@ public abstract class KeyGenerator implements Serializable {
    * Generate a Hoodie Key out of provided generic record.
    */
   public abstract HoodieKey getKey(GenericRecord record);
-}
+}
\ No newline at end of file
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java
index e790b46..bb2642c 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java
@@ -48,4 +48,4 @@ public class NonpartitionedKeyGenerator extends KeyGenerator {
     }
     return new HoodieKey(recordKey, EMPTY_PARTITION);
   }
-}
+}
\ No newline at end of file
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java
index e9b9396..a904860 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java
@@ -73,4 +73,12 @@ public class SimpleKeyGenerator extends KeyGenerator {
     }
     return recordKey;
   }
-}
+
+  public String getRecordKeyField() {
+    return recordKeyField;
+  }
+
+  public String getPartitionPathField() {
+    return partitionPathField;
+  }
+}
\ No newline at end of file
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java
index 7c34ef7..894df43 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java
@@ -149,7 +149,6 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
           "Unexpected type for partition field: " + partitionVal.getClass().getName());
       }
       DateTime timestamp = new DateTime(timeMs, outputDateTimeZone);
-
       return hiveStylePartitioning ? partitionPathField + "=" + timestamp.toString(partitionFormatter)
         : timestamp.toString(partitionFormatter);
     } catch (Exception e) {
diff --git a/hudi-spark/src/test/java/TestComplexKeyGenerator.java b/hudi-spark/src/test/java/TestComplexKeyGenerator.java
new file mode 100644
index 0000000..f0671fa
--- /dev/null
+++ b/hudi-spark/src/test/java/TestComplexKeyGenerator.java
@@ -0,0 +1,67 @@
+/*
+ * 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.
+ */
+
+import static junit.framework.TestCase.assertEquals;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.keygen.ComplexKeyGenerator;
+import org.junit.jupiter.api.Test;
+
+public class TestComplexKeyGenerator {
+
+  @Test
+  public void testSingleValueKeyGenerator() {
+    TypedProperties properties = new TypedProperties();
+    properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
+    properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "timestamp");
+    ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties);
+    assertEquals(compositeKeyGenerator.getRecordKeyFields().size(), 1);
+    assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 1);
+    HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
+    GenericRecord record = dataGenerator.generateGenericRecords(1).get(0);
+    String rowKey = record.get("_row_key").toString();
+    String partitionPath = record.get("timestamp").toString();
+    HoodieKey hoodieKey = compositeKeyGenerator.getKey(record);
+    assertEquals("_row_key:" + rowKey, hoodieKey.getRecordKey());
+    assertEquals(partitionPath, hoodieKey.getPartitionPath());
+  }
+
+  @Test
+  public void testMultipleValueKeyGenerator() {
+    TypedProperties properties = new TypedProperties();
+    properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key,timestamp");
+    properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "rider,driver");
+    ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties);
+    assertEquals(compositeKeyGenerator.getRecordKeyFields().size(), 2);
+    assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 2);
+    HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
+    GenericRecord record = dataGenerator.generateGenericRecords(1).get(0);
+    String rowKey =
+        "_row_key" + ComplexKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("_row_key").toString() + ","
+            + "timestamp" + ComplexKeyGenerator.DEFAULT_RECORD_KEY_SEPARATOR + record.get("timestamp").toString();
+    String partitionPath = record.get("rider").toString() + "/" + record.get("driver").toString();
+    HoodieKey hoodieKey = compositeKeyGenerator.getKey(record);
+    assertEquals(rowKey, hoodieKey.getRecordKey());
+    assertEquals(partitionPath, hoodieKey.getPartitionPath());
+  }
+
+}
\ No newline at end of file
diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml
index e320d67..742f5a1 100644
--- a/hudi-utilities/pom.xml
+++ b/hudi-utilities/pom.xml
@@ -131,6 +131,12 @@
       <artifactId>log4j</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <version>${slf4j.version}</version>
+    </dependency>
+
     <!-- Fasterxml -->
     <dependency>
       <groupId>com.fasterxml.jackson.module</groupId>
@@ -163,6 +169,10 @@
           <groupId>javax.servlet</groupId>
           <artifactId>*</artifactId>
         </exclusion>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-api</artifactId>
+        </exclusion>
       </exclusions>
     </dependency>
 
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
index 64b0238..1dffd07 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.utilities;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.AvroConversionUtils;
 import org.apache.hudi.client.HoodieWriteClient;
 import org.apache.hudi.client.WriteStatus;
@@ -35,6 +36,7 @@ import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.utilities.checkpointing.InitialCheckPointProvider;
 import org.apache.hudi.utilities.schema.SchemaProvider;
 import org.apache.hudi.utilities.sources.Source;
+import org.apache.hudi.utilities.sources.helpers.DFSPathSelector;
 import org.apache.hudi.utilities.transform.ChainedTransformer;
 import org.apache.hudi.utilities.transform.Transformer;
 
@@ -348,4 +350,15 @@ public class UtilHelpers {
       throw new HoodieException(String.format("%s table does not exists!", table));
     }
   }
+
+  public static DFSPathSelector createSourceSelector(String sourceSelectorClass, TypedProperties props,
+      Configuration conf) throws IOException {
+    try {
+      return (DFSPathSelector) ReflectionUtils.loadClass(sourceSelectorClass,
+          new Class<?>[]{TypedProperties.class, Configuration.class},
+          props, conf);
+    } catch (Throwable e) {
+      throw new IOException("Could not load source selector class " + sourceSelectorClass, e);
+    }
+  }
 }
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
index 41efc50..e7226fb 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
@@ -40,6 +40,7 @@ import org.apache.hudi.hive.HiveSyncTool;
 import org.apache.hudi.keygen.KeyGenerator;
 import org.apache.hudi.utilities.UtilHelpers;
 import org.apache.hudi.exception.HoodieDeltaStreamerException;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
 import org.apache.hudi.utilities.schema.DelegatingSchemaProvider;
 import org.apache.hudi.utilities.schema.RowBasedSchemaProvider;
 import org.apache.hudi.utilities.schema.SchemaProvider;
@@ -212,8 +213,8 @@ public class DeltaSync implements Serializable {
   /**
    * Run one round of delta sync and return new compaction instant if one got scheduled.
    */
-  public Option<String> syncOnce() throws Exception {
-    Option<String> scheduledCompaction = Option.empty();
+  public Pair<Option<String>, JavaRDD<WriteStatus>> syncOnce() throws Exception {
+    Pair<Option<String>, JavaRDD<WriteStatus>> result = null;
     HoodieDeltaStreamerMetrics metrics = new HoodieDeltaStreamerMetrics(getHoodieClientConfig(schemaProvider));
     Timer.Context overallTimerContext = metrics.getOverallTimerContext();
 
@@ -231,13 +232,13 @@ public class DeltaSync implements Serializable {
         setupWriteClient();
       }
 
-      scheduledCompaction = writeToSink(srcRecordsWithCkpt.getRight().getRight(),
+      result = writeToSink(srcRecordsWithCkpt.getRight().getRight(),
           srcRecordsWithCkpt.getRight().getLeft(), metrics, overallTimerContext);
     }
 
     // Clear persistent RDDs
     jssc.getPersistentRDDs().values().forEach(JavaRDD::unpersist);
-    return scheduledCompaction;
+    return result;
   }
 
   /**
@@ -248,7 +249,7 @@ public class DeltaSync implements Serializable {
    * of schemaProvider, checkpointStr and hoodieRecord
    * @throws Exception in case of any Exception
    */
-  private Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> readFromSource(
+  public Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> readFromSource(
       Option<HoodieTimeline> commitTimelineOpt) throws Exception {
     // Retrieve the previous round checkpoints, if any
     Option<String> resumeCheckpointStr = Option.empty();
@@ -355,8 +356,8 @@ public class DeltaSync implements Serializable {
    * @param overallTimerContext Timer Context
    * @return Option Compaction instant if one is scheduled
    */
-  private Option<String> writeToSink(JavaRDD<HoodieRecord> records, String checkpointStr,
-                                     HoodieDeltaStreamerMetrics metrics, Timer.Context overallTimerContext) throws Exception {
+  private Pair<Option<String>, JavaRDD<WriteStatus>> writeToSink(JavaRDD<HoodieRecord> records, String checkpointStr,
+      HoodieDeltaStreamerMetrics metrics, Timer.Context overallTimerContext) throws Exception {
 
     Option<String> scheduledCompactionInstant = Option.empty();
     // filter dupes if needed
@@ -413,7 +414,7 @@ public class DeltaSync implements Serializable {
         if (!isEmpty) {
           // Sync to hive if enabled
           Timer.Context hiveSyncContext = metrics.getHiveSyncTimerContext();
-          syncHive();
+          syncHiveIfNeeded();
           hiveSyncTimeMs = hiveSyncContext != null ? hiveSyncContext.stop() : 0;
         }
       } else {
@@ -438,7 +439,7 @@ public class DeltaSync implements Serializable {
     // Send DeltaStreamer Metrics
     metrics.updateDeltaStreamerMetrics(overallTimeMs, hiveSyncTimeMs);
 
-    return scheduledCompactionInstant;
+    return Pair.of(scheduledCompactionInstant, writeStatusRDD);
   }
 
   /**
@@ -472,15 +473,27 @@ public class DeltaSync implements Serializable {
   /**
    * Sync to Hive.
    */
-  private void syncHive() {
+  public void syncHiveIfNeeded() throws ClassNotFoundException {
     if (cfg.enableHiveSync) {
-      HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath, cfg.baseFileFormat);
-      LOG.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName + "). Hive metastore URL :"
-          + hiveSyncConfig.jdbcUrl + ", basePath :" + cfg.targetBasePath);
-      new HiveSyncTool(hiveSyncConfig, new HiveConf(conf, HiveConf.class), fs).syncHoodieTable();
+      syncHive();
     }
   }
 
+  public void syncHive() {
+    HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath, cfg.baseFileFormat);
+    LOG.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName + "). Hive metastore URL :"
+        + hiveSyncConfig.jdbcUrl + ", basePath :" + cfg.targetBasePath);
+    HiveConf hiveConf = new HiveConf(conf, HiveConf.class);
+    LOG.info("Hive Conf => " + hiveConf.getAllProperties().toString());
+    LOG.info("Hive Sync Conf => " + hiveSyncConfig.toString());
+    new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable();
+  }
+
+  public void syncHive(HiveConf conf) {
+    this.conf = conf;
+    syncHive();
+  }
+
   /**
    * Note that depending on configs and source-type, schemaProvider could either be eagerly or lazily created.
    * SchemaProvider creation is a precursor to HoodieWriteClient and AsyncCompactor creation. This method takes care of
@@ -558,4 +571,20 @@ public class DeltaSync implements Serializable {
       writeClient = null;
     }
   }
+
+  public FileSystem getFs() {
+    return fs;
+  }
+
+  public TypedProperties getProps() {
+    return props;
+  }
+
+  public Config getCfg() {
+    return cfg;
+  }
+
+  public Option<HoodieTimeline> getCommitTimelineOpt() {
+    return commitTimelineOpt;
+  }
 }
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
index 7290429..1562455 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
@@ -18,8 +18,9 @@
 
 package org.apache.hudi.utilities.deltastreamer;
 
-import org.apache.hudi.client.HoodieWriteClient;
 import org.apache.hudi.async.AbstractAsyncService;
+import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieTableType;
@@ -49,6 +50,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.sql.SparkSession;
 
@@ -83,9 +85,9 @@ public class HoodieDeltaStreamer implements Serializable {
 
   public static String CHECKPOINT_KEY = "deltastreamer.checkpoint.key";
 
-  private final transient Config cfg;
+  protected final transient Config cfg;
 
-  private transient DeltaSyncService deltaSyncService;
+  protected transient DeltaSyncService deltaSyncService;
 
   public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc) throws IOException {
     this(cfg, jssc, FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()),
@@ -435,11 +437,11 @@ public class HoodieDeltaStreamer implements Serializable {
           while (!isShutdownRequested()) {
             try {
               long start = System.currentTimeMillis();
-              Option<String> scheduledCompactionInstant = deltaSync.syncOnce();
-              if (scheduledCompactionInstant.isPresent()) {
-                LOG.info("Enqueuing new pending compaction instant (" + scheduledCompactionInstant + ")");
+              Pair<Option<String>, JavaRDD<WriteStatus>> scheduledCompactionInstantAndRDD = deltaSync.syncOnce();
+              if (scheduledCompactionInstantAndRDD.getLeft().isPresent()) {
+                LOG.info("Enqueuing new pending compaction instant (" + scheduledCompactionInstantAndRDD.getLeft() + ")");
                 asyncCompactService.enqueuePendingCompaction(new HoodieInstant(State.REQUESTED,
-                    HoodieTimeline.COMPACTION_ACTION, scheduledCompactionInstant.get()));
+                    HoodieTimeline.COMPACTION_ACTION, scheduledCompactionInstantAndRDD.getLeft().get()));
                 asyncCompactService.waitTillPendingCompactionsReducesTo(cfg.maxPendingCompactions);
               }
               long toSleepMs = cfg.minSyncIntervalSeconds * 1000 - (System.currentTimeMillis() - start);
@@ -623,4 +625,8 @@ public class HoodieDeltaStreamer implements Serializable {
       }, executor)).toArray(CompletableFuture[]::new)), executor);
     }
   }
+
+  public DeltaSyncService getDeltaSyncService() {
+    return deltaSyncService;
+  }
 }
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerMetrics.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerMetrics.java
index a054b27..38ec91e 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerMetrics.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerMetrics.java
@@ -23,15 +23,17 @@ import org.apache.hudi.metrics.Metrics;
 
 import com.codahale.metrics.Timer;
 
-public class HoodieDeltaStreamerMetrics {
+import java.io.Serializable;
+
+public class HoodieDeltaStreamerMetrics implements Serializable {
 
   private HoodieWriteConfig config;
   private String tableName;
 
   public String overallTimerName = null;
   public String hiveSyncTimerName = null;
-  private Timer overallTimer = null;
-  public Timer hiveSyncTimer = null;
+  private transient Timer overallTimer = null;
+  public transient Timer hiveSyncTimer = null;
 
   public HoodieDeltaStreamerMetrics(HoodieWriteConfig config) {
     this.config = config;
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java
index 9c0be88..7fd23b8 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java
@@ -71,11 +71,11 @@ public final class SourceFormatAdapter {
                     // pass in the schema for the Row-to-Avro conversion
                     // to avoid nullability mismatch between Avro schema and Row schema
                     ? AvroConversionUtils.createRdd(
-                        rdd, r.getSchemaProvider().getSourceSchema(),
-                        HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD()
+                    rdd, r.getSchemaProvider().getSourceSchema(),
+                    HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD()
                     : AvroConversionUtils.createRdd(
                         rdd, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD()
-                ))
+            ))
             .orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider());
       }
       default:
@@ -116,4 +116,4 @@ public final class SourceFormatAdapter {
         throw new IllegalArgumentException("Unknown source type (" + source.getSourceType() + ")");
     }
   }
-}
+}
\ No newline at end of file
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroDFSSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroDFSSource.java
index a2f7df9..b5ce96f 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroDFSSource.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroDFSSource.java
@@ -21,6 +21,7 @@ package org.apache.hudi.utilities.sources;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.utilities.UtilHelpers;
 import org.apache.hudi.utilities.schema.SchemaProvider;
 import org.apache.hudi.utilities.sources.helpers.DFSPathSelector;
 
@@ -33,6 +34,8 @@ import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.sql.SparkSession;
 
+import java.io.IOException;
+
 /**
  * DFS Source that reads avro data.
  */
@@ -41,9 +44,11 @@ public class AvroDFSSource extends AvroSource {
   private final DFSPathSelector pathSelector;
 
   public AvroDFSSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
-      SchemaProvider schemaProvider) {
+      SchemaProvider schemaProvider) throws IOException {
     super(props, sparkContext, sparkSession, schemaProvider);
-    this.pathSelector = new DFSPathSelector(props, sparkContext.hadoopConfiguration());
+    this.pathSelector = UtilHelpers
+        .createSourceSelector(DFSPathSelector.class.getName(), props, sparkContext
+        .hadoopConfiguration());
   }
 
   @Override
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java
index cb0100a..59263e4 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/DFSPathSelector.java
@@ -32,6 +32,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -42,18 +44,20 @@ import java.util.stream.Collectors;
 
 public class DFSPathSelector {
 
+  protected static volatile Logger log = LogManager.getLogger(DFSPathSelector.class);
+
   /**
    * Configs supported.
    */
-  static class Config {
+  public static class Config {
 
-    private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root";
+    public static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root";
   }
 
-  private static final List<String> IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_");
+  protected static final List<String> IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_");
 
-  private final transient FileSystem fs;
-  private final TypedProperties props;
+  protected final transient FileSystem fs;
+  protected final TypedProperties props;
 
   public DFSPathSelector(TypedProperties props, Configuration hadoopConf) {
     DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
@@ -66,6 +70,7 @@ public class DFSPathSelector {
 
     try {
       // obtain all eligible files under root folder.
+      log.info("Root path => " + props.getString(Config.ROOT_INPUT_PATH_PROP) + " source limit => " + sourceLimit);
       List<FileStatus> eligibleFiles = new ArrayList<>();
       RemoteIterator<LocatedFileStatus> fitr =
           fs.listFiles(new Path(props.getString(Config.ROOT_INPUT_PATH_PROP)), true);
@@ -79,7 +84,6 @@ public class DFSPathSelector {
       }
       // sort them by modification time.
       eligibleFiles.sort(Comparator.comparingLong(FileStatus::getModificationTime));
-
       // Filter based on checkpoint & input size, if needed
       long currentBytes = 0;
       long maxModificationTime = Long.MIN_VALUE;
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestCsvDFSSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestCsvDFSSource.java
index 8fe99d8..7b8eead 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestCsvDFSSource.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestCsvDFSSource.java
@@ -42,7 +42,7 @@ public class TestCsvDFSSource extends AbstractDFSSourceTestBase {
     this.fileSuffix = ".json";
     this.useFlattenedSchema = true;
     this.schemaProvider = new FilebasedSchemaProvider(
-        Helpers.setupSchemaOnDFS("source-flattened.avsc"), jsc);
+        Helpers.setupSchemaOnDFS("delta-streamer-config", "source-flattened.avsc"), jsc);
   }
 
   @Override
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java
index e8a20d8..4fbbe5b 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.utilities.testutils;
 
+import java.io.FileInputStream;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieTableType;
@@ -189,6 +190,17 @@ public class UtilitiesTestBase {
       return sb.toString();
     }
 
+    public static String readFileFromAbsolutePath(String absolutePathForResource) throws IOException {
+      BufferedReader reader =
+          new BufferedReader(new InputStreamReader(new FileInputStream(absolutePathForResource)));
+      StringBuffer sb = new StringBuffer();
+      String line;
+      while ((line = reader.readLine()) != null) {
+        sb.append(line + "\n");
+      }
+      return sb.toString();
+    }
+
     public static void copyToDFS(String testResourcePath, FileSystem fs, String targetPath) throws IOException {
       PrintStream os = new PrintStream(fs.create(new Path(targetPath), true));
       os.print(readFile(testResourcePath));
@@ -196,6 +208,14 @@ public class UtilitiesTestBase {
       os.close();
     }
 
+    public static void copyToDFSFromAbsolutePath(String absolutePathForResource, FileSystem fs, String targetPath)
+        throws IOException {
+      PrintStream os = new PrintStream(fs.create(new Path(targetPath), true));
+      os.print(readFileFromAbsolutePath(absolutePathForResource));
+      os.flush();
+      os.close();
+    }
+
     public static void savePropsToDFS(TypedProperties props, FileSystem fs, String targetPath) throws IOException {
       String[] lines = props.keySet().stream().map(k -> String.format("%s=%s", k, props.get(k))).toArray(String[]::new);
       saveStringsToDFS(lines, fs, targetPath);
@@ -258,11 +278,18 @@ public class UtilitiesTestBase {
     }
 
     public static TypedProperties setupSchemaOnDFS() throws IOException {
-      return setupSchemaOnDFS("source.avsc");
+      return setupSchemaOnDFS("delta-streamer-config", "source.avsc");
+    }
+
+    public static TypedProperties setupSchemaOnDFS(String scope, String filename) throws IOException {
+      UtilitiesTestBase.Helpers.copyToDFS(scope + "/" + filename, dfs, dfsBasePath + "/" + filename);
+      TypedProperties props = new TypedProperties();
+      props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/" + filename);
+      return props;
     }
 
-    public static TypedProperties setupSchemaOnDFS(String filename) throws IOException {
-      UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/" + filename, dfs, dfsBasePath + "/" + filename);
+    public static TypedProperties setupSchemaOnDFSWithAbsoluteScope(String scope, String filename) throws IOException {
+      UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(scope + "/" + filename, dfs, dfsBasePath + "/" + filename);
       TypedProperties props = new TypedProperties();
       props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/" + filename);
       return props;
@@ -278,7 +305,7 @@ public class UtilitiesTestBase {
     }
 
     public static List<GenericRecord> toGenericRecords(List<HoodieRecord> hoodieRecords) {
-      List<GenericRecord> records = new ArrayList<GenericRecord>();
+      List<GenericRecord> records = new ArrayList<>();
       for (HoodieRecord hoodieRecord : hoodieRecords) {
         records.add(toGenericRecord(hoodieRecord));
       }
diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/complex-source.avsc b/hudi-utilities/src/test/resources/delta-streamer-config/complex-source.avsc
new file mode 100644
index 0000000..56a3bf0
--- /dev/null
+++ b/hudi-utilities/src/test/resources/delta-streamer-config/complex-source.avsc
@@ -0,0 +1,466 @@
+/*
+ * 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.
+ */
+ {
+    "name": "COMPLEX",
+    "fields": [
+        {
+            "default": null,
+            "type": [
+                "null",
+                {
+                    "items": "string",
+                    "type": "array"
+                }
+            ],
+            "name": "array_of_string_fields1"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field1"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field2"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field3"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field4"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field5"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "boolean"
+            ],
+            "name": "boolean_field1"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field6"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field7"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field8"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field9"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field10"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field11"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field12"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "double"
+            ],
+            "name": "double_field1"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field13"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field1"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field14"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field2"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                {
+                    "items": {
+                        "fields": [
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "string"
+                                ],
+                                "name": "string_field15"
+                            },
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "string"
+                                ],
+                                "name": "string_field16"
+                            },
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "string"
+                                ],
+                                "name": "string_field17"
+                            },
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "long"
+                                ],
+                                "name": "long_field3"
+                            },
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "long"
+                                ],
+                                "name": "long_field4"
+                            },
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "double"
+                                ],
+                                "name": "double_field2"
+                            },
+                            {
+                                "default": null,
+                                "type": [
+                                    "null",
+                                    "double"
+                                ],
+                                "name": "double_field3"
+                            }
+                        ],
+                        "type": "record",
+                        "name": "record_field1"
+                    },
+                    "type": "array"
+                }
+            ],
+            "name": "record_name1"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field18"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field5"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "double"
+            ],
+            "name": "double_field4"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "double"
+            ],
+            "name": "double_field5"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field19"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field6"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field20"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field7"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "double"
+            ],
+            "name": "double_field6"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field21"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field22"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field23"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field8"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "double"
+            ],
+            "name": "double_field7"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field24"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field10"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field25"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field26"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field11"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "boolean"
+            ],
+            "name": "boolean_field3"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field12"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "double"
+            ],
+            "name": "double_field8"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "long"
+            ],
+            "name": "long_field13"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field27"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field28"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field29"
+        },
+        {
+            "default": null,
+            "type": [
+                "null",
+                "string"
+            ],
+            "name": "string_field30"
+        }
+    ],
+    "type": "record"
+}
\ No newline at end of file
diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml
new file mode 100644
index 0000000..9ce67fd
--- /dev/null
+++ b/packaging/hudi-integ-test-bundle/pom.xml
@@ -0,0 +1,565 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<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/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>hudi</artifactId>
+    <groupId>org.apache.hudi</groupId>
+    <version>0.6.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>hudi-integ-test-bundle</artifactId>
+  <packaging>jar</packaging>
+
+  <properties>
+    <checkstyle.skip>true</checkstyle.skip>
+    <main.basedir>${project.parent.basedir}</main.basedir>
+  </properties>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.jacoco</groupId>
+        <artifactId>jacoco-maven-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>${maven-shade-plugin.version}</version>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+            <configuration>
+              <createSourcesJar>true</createSourcesJar>
+              <dependencyReducedPomLocation>${project.build.directory}/dependency-reduced-pom.xml
+              </dependencyReducedPomLocation>
+              <transformers>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer">
+                </transformer>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer">
+                  <addHeader>true</addHeader>
+                </transformer>
+                <transformer implementation="org.apache.maven.plugins.shade.resource.IncludeResourceTransformer">
+                  <resource>META-INF/LICENSE</resource>
+                  <file>target/classes/META-INF/LICENSE</file>
+                </transformer>
+              </transformers>
+              <artifactSet>
+                <includes>
+                  <include>commons-codec:commons-codec</include>
+                  <include>commons-dbcp:commons-dbcp</include>
+                  <include>commons-lang:commons-lang</include>
+                  <include>commons-pool:commons-pool</include>
+                  <include>org.apache.hudi:hudi-common</include>
+                  <include>org.apache.hudi:hudi-client</include>
+                  <include>org.apache.hudi:hudi-utilities_${scala.binary.version}</include>
+                  <include>org.apache.hudi:hudi-spark_${scala.binary.version}</include>
+                  <include>org.apache.hudi:hudi-hive-sync</include>
+                  <include>org.apache.hudi:hudi-hadoop-mr</include>
+                  <include>org.apache.hudi:hudi-timeline-service</include>
+                  <include>org.apache.hudi:hudi-integ-test</include>
+                  <include>com.beust:jcommander</include>
+                  <include>com.twitter:bijection-avro_${scala.binary.version}</include>
+                  <include>com.twitter:bijection-core_${scala.binary.version}</include>
+                  <include>org.apache.parquet:parquet-avro</include>
+                  <include>com.twitter:parquet-avro</include>
+                  <include>com.twitter.common:objectsize</include>
+                  <include>io.confluent:kafka-avro-serializer</include>
+                  <include>io.confluent:common-config</include>
+                  <include>io.confluent:common-utils</include>
+                  <include>io.confluent:kafka-schema-registry-client</include>
+                  <include>io.dropwizard.metrics:metrics-core</include>
+                  <include>io.dropwizard.metrics:metrics-graphite</include>
+                  <include>org.apache.spark:spark-streaming-kafka-0-10_${scala.binary.version}</include>
+                  <include>org.apache.kafka:kafka_${scala.binary.version}</include>
+                  <include>com.101tec:zkclient</include>
+                  <include>org.apache.kafka:kafka-clients</include>
+                  <include>org.apache.hive:hive-common</include>
+                  <include>org.apache.hive:hive-service</include>
+                  <include>org.apache.hive:hive-metastore</include>
+                  <include>org.apache.hive:hive-jdbc</include>
+                  <include>org.apache.hive:hive-exec</include>
+                  <include>com.esotericsoftware:kryo-shaded</include>
+                  <include>org.objenesis:objenesis</include>
+                  <include>com.esotericsoftware:minlog</include>
+                  <include>com.yammer.metrics:metrics-core</include>
+                  <include>org.apache.thrift:libfb303</include>
+                  <include>org.apache.thrift:libthrift</include>
+                  <include>com.fasterxml.jackson.dataformat:jackson-dataformat-yaml</include>
+                </includes>
+              </artifactSet>
+              <relocations>
+                <relocation>
+                  <pattern>com.beust.jcommander.</pattern>
+                  <shadedPattern>org.apache.hudi.com.beust.jcommander.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.commons.dbcp.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.commons.dbcp.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.commons.lang.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.commons.lang.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.commons.pool.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.commons.pool.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hive.jdbc.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hive.jdbc.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hive.common.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hive.common.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hadoop.hive.common.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hadoop.hive.common.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hadoop.hive.shims.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hadoop.hive.shims.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hadoop.hive.thrift.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hadoop.hive.thrift.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hadoop.hive.serde2.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hadoop.hive.serde2.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hadoop.hive.io.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hadoop.hive.io.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hive.service.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hive.service.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hadoop.hive.service.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hadoop_hive.service.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hive.org.apache.thrift.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hive.org.apache.thrift.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.thrift.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.thrift.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>com.facebook.fb303.</pattern>
+                  <shadedPattern>org.apache.hudi.com.facebook.fb303.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hive.jdbc.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.hive.jdbc.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>com.esotericsoftware.kryo.</pattern>
+                  <shadedPattern>org.apache.hudi.com.esotericsoftware.kryo.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.objenesis.</pattern>
+                  <shadedPattern>org.apache.hudi.org.objenesis.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>com.esotericsoftware.minlog.</pattern>
+                  <shadedPattern>org.apache.hudi.com.esotericsoftware.minlog.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>com.codahale.metrics.</pattern>
+                  <shadedPattern>org.apache.hudi.com.codahale.metrics.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.commons.codec.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.commons.codec.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>com.fasterxml.jackson.dataformat.</pattern>
+                  <shadedPattern>org.apache.hudi.com.fasterxml.jackson.dataformat.</shadedPattern>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.parquet.avro.</pattern>
+                  <shadedPattern>org.apache.hudi.org.apache.parquet.avro.</shadedPattern>
+                </relocation>
+              </relocations>
+              <filters>
+                <filter>
+                  <artifact>*:*</artifact>
+                  <excludes>
+                    <exclude>META-INF/*.SF</exclude>
+                    <exclude>META-INF/*.DSA</exclude>
+                    <exclude>META-INF/*.RSA</exclude>
+                    <!-- Use this jar's NOTICE and license file -->
+                    <exclude>META-INF/NOTICE*</exclude>
+                    <exclude>META-INF/LICENSE*</exclude>
+                  </excludes>
+                </filter>
+              </filters>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+      </resource>
+      <resource>
+        <directory>src/test/resources</directory>
+      </resource>
+    </resources>
+  </build>
+
+  <repositories>
+    <repository>
+      <id>confluent</id>
+      <url>https://packages.confluent.io/maven/</url>
+    </repository>
+  </repositories>
+
+  <dependencies>
+    <dependency>
+      <groupId>io.javalin</groupId>
+      <artifactId>javalin</artifactId>
+      <version>2.4.0</version>
+    </dependency>
+
+    <dependency>
+      <groupId>io.dropwizard.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.module</groupId>
+      <artifactId>jackson-module-scala_${scala.binary.version}</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.fasterxml.jackson.dataformat</groupId>
+      <artifactId>jackson-dataformat-yaml</artifactId>
+      <version>2.7.4</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-common</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-hive-sync</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <type>test-jar</type>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-spark_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <classifier>tests</classifier>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <classifier>tests</classifier>
+      <exclusions>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet.jsp</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>${hive.groupid}</groupId>
+      <artifactId>hive-exec</artifactId>
+      <version>${hive.version}</version>
+      <classifier>${hive.exec.classifier}</classifier>
+    </dependency>
+
+    <dependency>
+      <groupId>${hive.groupid}</groupId>
+      <artifactId>hive-jdbc</artifactId>
+      <version>${hive.version}</version>
+      <classifier>standalone</classifier>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>${hive.groupid}</groupId>
+      <artifactId>hive-jdbc</artifactId>
+      <version>${hive.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>org.slf4j</groupId>
+          <artifactId>slf4j-api</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>${hive.groupid}</groupId>
+      <artifactId>hive-common</artifactId>
+      <version>${hive.version}</version>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-hive-sync</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-client</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-utilities_${scala.binary.version}</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-integ-test</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-client</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>commons-codec</groupId>
+      <artifactId>commons-codec</artifactId>
+      <version>1.4</version>
+    </dependency>
+
+    <dependency>
+      <groupId>commons-dbcp</groupId>
+      <artifactId>commons-dbcp</artifactId>
+      <version>1.4</version>
+    </dependency>
+
+    <dependency>
+      <groupId>commons-pool</groupId>
+      <artifactId>commons-pool</artifactId>
+      <version>1.4</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpcore</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+      <version>${slf4j.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-mapreduce-client-common</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>com.yammer.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+      <version>2.2.0</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+      <scope>provided</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-streaming-kafka-0-10_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+    </dependency>
+
+    <!-- Used for SQL templating -->
+    <dependency>
+      <groupId>org.antlr</groupId>
+      <artifactId>stringtemplate</artifactId>
+      <version>4.0.2</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.beust</groupId>
+      <artifactId>jcommander</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>1.10.19</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro-mapred</artifactId>
+      <version>1.7.7</version>
+    </dependency>
+
+    <!-- Parquet -->
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-avro</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-hadoop</artifactId>
+      <version>${parquet.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>bijection-avro_${scala.binary.version}</artifactId>
+      <version>0.9.3</version>
+    </dependency>
+
+    <dependency>
+      <groupId>io.confluent</groupId>
+      <artifactId>kafka-avro-serializer</artifactId>
+      <version>3.0.0</version>
+    </dependency>
+
+    <dependency>
+      <groupId>io.confluent</groupId>
+      <artifactId>common-config</artifactId>
+      <version>3.0.0</version>
+    </dependency>
+
+    <dependency>
+      <groupId>io.confluent</groupId>
+      <artifactId>common-utils</artifactId>
+      <version>3.0.0</version>
+    </dependency>
+
+    <dependency>
+      <groupId>io.confluent</groupId>
+      <artifactId>kafka-schema-registry-client</artifactId>
+      <version>3.0.0</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.thrift</groupId>
+      <artifactId>libfb303</artifactId>
+      <version>0.9.3</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.thrift</groupId>
+      <artifactId>libthrift</artifactId>
+      <version>0.9.3</version>
+    </dependency>
+
+  </dependencies>
+</project>
+
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/packaging/hudi-integ-test-bundle/src/main/java/org/apache/hudi/testsuite/bundle/Main.java
similarity index 55%
copy from hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
copy to packaging/hudi-integ-test-bundle/src/main/java/org/apache/hudi/testsuite/bundle/Main.java
index 7dde326..268d2d0 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/packaging/hudi-integ-test-bundle/src/main/java/org/apache/hudi/testsuite/bundle/Main.java
@@ -16,28 +16,21 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
+package org.apache.hudi.testsuite.bundle;
 
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieKey;
-
-import org.apache.avro.generic.GenericRecord;
-
-import java.io.Serializable;
+import org.apache.hudi.common.util.ReflectionUtils;
 
 /**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
+ * A simple main class to dump all classes loaded in current classpath
+ *
+ * This is a workaround for generating sources and javadoc jars for packaging modules. The maven plugins for generating
+ * javadoc and sources plugins do not generate corresponding jars if there are no source files.
+ *
+ * This class does not have anything to do with Hudi but is there to keep mvn javadocs/source plugin happy.
  */
-public abstract class KeyGenerator implements Serializable {
-
-  protected transient TypedProperties config;
+public class Main {
 
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
+  public static void main(String[] args) {
+    ReflectionUtils.getTopLevelClassesInClasspath(Main.class).forEach(System.out::println);
   }
-
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
 }
diff --git a/pom.xml b/pom.xml
index 05907f3..76ae34a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -51,6 +51,7 @@
     <module>packaging/hudi-timeline-server-bundle</module>
     <module>docker/hoodie/hadoop</module>
     <module>hudi-integ-test</module>
+    <module>packaging/hudi-integ-test-bundle</module>
     <module>hudi-examples</module>
   </modules>
 
@@ -89,7 +90,7 @@
     <junit.platform.version>1.7.0-M1</junit.platform.version>
     <mockito.jupiter.version>3.3.3</mockito.jupiter.version>
     <log4j.version>1.2.17</log4j.version>
-    <slf4j.version>1.7.5</slf4j.version>
+    <slf4j.version>1.7.15</slf4j.version>
     <joda.version>2.9.9</joda.version>
     <hadoop.version>2.7.3</hadoop.version>
     <hive.groupid>org.apache.hive</hive.groupid>
@@ -995,6 +996,7 @@
               <excludes>
                 <exclude>**/*FunctionalTestSuite.java</exclude>
                 <exclude>**/IT*.java</exclude>
+                <exclude>**/testsuite/**/Test*.java</exclude>
               </excludes>
             </configuration>
           </plugin>
@@ -1096,6 +1098,7 @@
             <configuration combine.self="override">
               <skip>${skipITs}</skip>
               <includes>
+                <include>**/testsuite/**/Test*.java</include>
                 <include>**/IT*.java</include>
               </includes>
             </configuration>