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 2019/12/10 22:47:45 UTC

[incubator-hudi] branch hudi_test_suite_refactor updated (504c2cc -> c82d6d9)

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

nagarwal pushed a change to branch hudi_test_suite_refactor
in repository https://gitbox.apache.org/repos/asf/incubator-hudi.git.


 discard 504c2cc  Fixing some unit tests
    omit 0c2ed53  fixing build issues due to javax servlet
    omit 07b4c12  Adressing CR comments part 1
    omit c3f8c8c      Hudi Test Suite     - Flexible schema payload generation     - Different types of workload generation such as inserts, upserts etc     - Post process actions to perform validations     - Interoperability of test suite to use HoodieWriteClient and HoodieDeltaStreamer so both code paths can be tested     - Custom workload sequence generator     - Ability to perform parallel operations, such as upsert and compaction
     new c82d6d9  Hudi Test Suite     - Flexible schema payload generation     - Different types of workload generation such as inserts, upserts etc     - Post process actions to perform validations     - Interoperability of test suite to use HoodieWriteClient and HoodieDeltaStreamer so both code paths can be tested     - Custom workload sequence generator     - Ability to perform parallel operations, such as upsert and compaction

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (504c2cc)
            \
             N -- N -- N   refs/heads/hudi_test_suite_refactor (c82d6d9)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:


[incubator-hudi] 01/01: Hudi Test Suite - Flexible schema payload generation - Different types of workload generation such as inserts, upserts etc - Post process actions to perform validations - Interoperability of test suite to use HoodieWriteClient and HoodieDeltaStreamer so both code paths can be tested - Custom workload sequence generator - Ability to perform parallel operations, such as upsert and compaction

Posted by na...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

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

    Hudi Test Suite
        - Flexible schema payload generation
        - Different types of workload generation such as inserts, upserts etc
        - Post process actions to perform validations
        - Interoperability of test suite to use HoodieWriteClient and HoodieDeltaStreamer so both code paths can be tested
        - Custom workload sequence generator
        - Ability to perform parallel operations, such as upsert and compaction
---
 docker/demo/config/bench/complex-source.avsc       | 449 ++++++++++++++++
 .../config/bench/complex-workflow-dag-cow.yaml     |  71 +++
 .../config/bench/complex-workflow-dag-mor.yaml     | 104 ++++
 docker/demo/config/bench/source.avsc               |  34 ++
 docker/demo/config/bench/target.avsc               |  37 ++
 docker/demo/config/bench/test-source.properties    |  12 +
 docker/hoodie/hadoop/hive_base/Dockerfile          |   1 +
 docker/hoodie/hadoop/hive_base/pom.xml             |   3 +
 hudi-bench/pom.xml                                 | 357 +++++++++++++
 hudi-bench/prepare_integration_suite.sh            | 112 ++++
 .../apache/hudi/bench/DFSDeltaWriterAdapter.java   |  68 +++
 .../apache/hudi/bench/DFSSparkAvroDeltaWriter.java |  25 +-
 .../org/apache/hudi/bench/DeltaInputFormat.java    |  23 +-
 .../org/apache/hudi/bench/DeltaOutputType.java     |  23 +-
 .../org/apache/hudi/bench/DeltaWriterAdapter.java  |  28 +-
 .../org/apache/hudi/bench/DeltaWriterFactory.java  |  59 +++
 .../hudi/bench/configuration/DFSDeltaConfig.java   |  73 +++
 .../hudi/bench/configuration/DeltaConfig.java      | 258 ++++++++++
 .../hudi/bench/converter/UpdateConverter.java      |  54 ++
 .../java/org/apache/hudi/bench/dag/DagUtils.java   | 218 ++++++++
 .../apache/hudi/bench/dag/ExecutionContext.java    |  53 ++
 .../org/apache/hudi/bench/dag/WorkflowDag.java     |  26 +-
 .../hudi/bench/dag/WorkflowDagGenerator.java       |  74 +++
 .../hudi/bench/dag/nodes/BulkInsertNode.java       |  29 +-
 .../org/apache/hudi/bench/dag/nodes/CleanNode.java |  23 +-
 .../apache/hudi/bench/dag/nodes/CompactNode.java   |  47 ++
 .../org/apache/hudi/bench/dag/nodes/DagNode.java   | 125 +++++
 .../apache/hudi/bench/dag/nodes/HiveQueryNode.java |  77 +++
 .../apache/hudi/bench/dag/nodes/HiveSyncNode.java  |  46 ++
 .../apache/hudi/bench/dag/nodes/InsertNode.java    |  65 +++
 .../apache/hudi/bench/dag/nodes/RollbackNode.java  |  48 ++
 .../hudi/bench/dag/nodes/ScheduleCompactNode.java  |  55 ++
 .../hudi/bench/dag/nodes/SparkSQLQueryNode.java    |  61 +++
 .../apache/hudi/bench/dag/nodes/UpsertNode.java    |  56 ++
 .../apache/hudi/bench/dag/nodes/ValidateNode.java  |  48 ++
 .../hudi/bench/dag/scheduler/DagScheduler.java     | 100 ++++
 .../hudi/bench/generator/DeltaGenerator.java       | 237 +++++++++
 .../FlexibleSchemaRecordGenerationIterator.java    |  70 +++
 .../GenericRecordFullPayloadGenerator.java         | 245 +++++++++
 .../GenericRecordFullPayloadSizeEstimator.java     | 117 +++++
 .../GenericRecordPartialPayloadGenerator.java      |  84 +++
 .../generator/LazyRecordGeneratorIterator.java     |  29 +-
 .../bench/generator/UpdateGeneratorIterator.java   |  56 ++
 .../bench/helpers/DFSTestSuitePathSelector.java    |  92 ++++
 .../hudi/bench/helpers/HiveServiceProvider.java    |  64 +++
 .../hudi/bench/job/HoodieDeltaStreamerWrapper.java |  69 +++
 .../apache/hudi/bench/job/HoodieTestSuiteJob.java  | 184 +++++++
 .../hudi/bench/reader/DFSAvroDeltaInputReader.java |  90 ++++
 .../hudi/bench/reader/DFSDeltaInputReader.java     |  98 ++++
 .../bench/reader/DFSHoodieDatasetInputReader.java  | 286 +++++++++++
 .../bench/reader/DFSParquetDeltaInputReader.java   |  90 ++++
 .../apache/hudi/bench/reader/DeltaInputReader.java |  62 +++
 .../apache/hudi/bench/reader/SparkBasedReader.java |  68 +++
 .../hudi/bench/writer/AvroDeltaInputWriter.java    | 121 +++++
 .../apache/hudi/bench/writer/DeltaInputWriter.java |  36 +-
 .../org/apache/hudi/bench/writer/DeltaWriter.java  | 184 +++++++
 .../hudi/bench/writer/FileDeltaInputWriter.java    |  24 +-
 .../bench/writer/SparkAvroDeltaInputWriter.java    |  65 +++
 .../org/apache/hudi/bench/writer/WriteStats.java   |  72 +++
 .../hudi/bench/TestDFSDeltaWriterAdapter.java      | 130 +++++
 .../hudi/bench/TestFileDeltaInputWriter.java       | 143 ++++++
 .../bench/configuration/TestWorkflowBuilder.java   |  76 +++
 .../hudi/bench/converter/TestUpdateConverter.java  |  93 ++++
 .../org/apache/hudi/bench/dag/TestComplexDag.java  |  78 +++
 .../org/apache/hudi/bench/dag/TestDagUtils.java    |  90 ++++
 .../org/apache/hudi/bench/dag/TestHiveSyncDag.java |  55 ++
 .../apache/hudi/bench/dag/TestInsertOnlyDag.java   |  50 ++
 .../apache/hudi/bench/dag/TestInsertUpsertDag.java |  51 ++
 .../TestGenericRecordPayloadEstimator.java         |  51 ++
 .../TestGenericRecordPayloadGenerator.java         | 122 +++++
 .../bench/generator/TestWorkloadGenerator.java     | 372 ++++++++++++++
 .../hudi/bench/job/TestHoodieTestSuiteJob.java     | 203 ++++++++
 .../bench/reader/TestDFSAvroDeltaInputReader.java  |  71 +++
 .../reader/TestDFSHoodieDatasetInputReader.java    | 122 +++++
 .../org/apache/hudi/bench/utils/TestUtils.java     |  61 +++
 .../apache/hudi/bench/writer/TestDeltaWriter.java  | 114 +++++
 .../resources/hudi-bench-config/base.properties    |  21 +
 .../hudi-bench-config/complex-source.avsc          | 449 ++++++++++++++++
 .../complex-workflow-dag-cow.yaml                  |  82 +++
 .../test/resources/hudi-bench-config/source.avsc   |  34 ++
 .../test/resources/hudi-bench-config/target.avsc   |  37 ++
 .../test/resources/log4j-surefire-quiet.properties |  24 +
 .../src/test/resources/log4j-surefire.properties   |  26 +
 .../hudi/common/HoodieTestDataGenerator.java       |   9 +
 .../java/org/apache/hudi/hive/HiveSyncTool.java    |   2 +
 .../org/apache/hudi/hive/HoodieHiveClient.java     |   4 +
 .../org/apache/hudi/hive/util/HiveTestService.java |   3 +-
 hudi-spark/pom.xml                                 |  12 +
 .../java/org/apache/hudi/ComplexKeyGenerator.java  |  28 +-
 .../main/java/org/apache/hudi/DataSourceUtils.java |   2 +-
 .../org/apache/hudi/EmptyHoodieRecordPayload.java  |   3 +-
 .../main/java/org/apache/hudi/KeyGenerator.java    |  13 +
 .../apache/hudi/NonpartitionedKeyGenerator.java    |   2 +-
 .../main/java/org/apache/hudi/QuickstartUtils.java |  57 ++-
 .../java/org/apache/hudi/SimpleKeyGenerator.java   |  32 +-
 .../src/test/java/TestComplexKeyGenerator.java     |  67 +++
 .../org/apache/hudi/utilities/UtilHelpers.java     |  13 +
 .../apache/hudi/utilities/converter/Converter.java |  26 +-
 .../hudi/utilities/deltastreamer/DeltaSync.java    |  52 +-
 .../deltastreamer/HoodieDeltaStreamer.java         |  12 +-
 .../deltastreamer/HoodieDeltaStreamerMetrics.java  |   7 +-
 .../keygen/TimestampBasedKeyGenerator.java         |   5 +-
 .../hudi/utilities/sources/AvroDFSSource.java      |   8 +-
 .../utilities/sources/helpers/DFSPathSelector.java |  16 +-
 .../hudi/utilities/TestHoodieDeltaStreamer.java    |  10 +-
 .../apache/hudi/utilities/UtilitiesTestBase.java   |  12 +-
 .../hudi/utilities/sources/TestDFSSource.java      |   2 +-
 .../hudi/utilities/sources/TestKafkaSource.java    |   2 +-
 .../delta-streamer-config/complex-source.avsc      | 449 ++++++++++++++++
 packaging/hudi-bench-bundle/pom.xml                | 565 +++++++++++++++++++++
 .../java/org/apache/hudi/bench/bundle/Main.java    |  27 +-
 pom.xml                                            |  49 +-
 112 files changed, 9138 insertions(+), 291 deletions(-)

diff --git a/docker/demo/config/bench/complex-source.avsc b/docker/demo/config/bench/complex-source.avsc
new file mode 100644
index 0000000..8761306
--- /dev/null
+++ b/docker/demo/config/bench/complex-source.avsc
@@ -0,0 +1,449 @@
+{
+    "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/docker/demo/config/bench/complex-workflow-dag-cow.yaml b/docker/demo/config/bench/complex-workflow-dag-cow.yaml
new file mode 100644
index 0000000..9cd092d
--- /dev/null
+++ b/docker/demo/config/bench/complex-workflow-dag-cow.yaml
@@ -0,0 +1,71 @@
+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.test_table group by `_row_key` having count(*) > 1"
+      result1: 0
+      query2: "select count(*) from testdb.test_table"
+      result2: 3100
+  type: HiveQueryNode
+  deps: second_upsert
\ No newline at end of file
diff --git a/docker/demo/config/bench/complex-workflow-dag-mor.yaml b/docker/demo/config/bench/complex-workflow-dag-mor.yaml
new file mode 100644
index 0000000..311b5d3
--- /dev/null
+++ b/docker/demo/config/bench/complex-workflow-dag-mor.yaml
@@ -0,0 +1,104 @@
+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.test_table group by `_row_key` having count(*) > 1"
+      result1: 0
+      query2: "select count(*) from testdb.test_table"
+      result2: 3100
+      query3: "select count(*) from testdb.test_table_rt group by `_row_key` having count(*) > 1"
+      result3: 0
+      query4: "select count(*) from testdb.test_table_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.test_table group by `_row_key` having count(*) > 1"
+      result1: 0
+      query2: "select count(*) from testdb.test_table"
+      result2: 2210
+  type: HiveQueryNode
+  deps: second_upsert
\ No newline at end of file
diff --git a/docker/demo/config/bench/source.avsc b/docker/demo/config/bench/source.avsc
new file mode 100644
index 0000000..4c7892d
--- /dev/null
+++ b/docker/demo/config/bench/source.avsc
@@ -0,0 +1,34 @@
+{
+  "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/docker/demo/config/bench/target.avsc b/docker/demo/config/bench/target.avsc
new file mode 100644
index 0000000..d2d4103
--- /dev/null
+++ b/docker/demo/config/bench/target.avsc
@@ -0,0 +1,37 @@
+{
+  "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/bench/test-source.properties b/docker/demo/config/bench/test-source.properties
new file mode 100644
index 0000000..4c46c65
--- /dev/null
+++ b/docker/demo/config/bench/test-source.properties
@@ -0,0 +1,12 @@
+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 b8fde36..1273c7e 100644
--- a/docker/hoodie/hadoop/hive_base/Dockerfile
+++ b/docker/hoodie/hadoop/hive_base/Dockerfile
@@ -56,6 +56,7 @@ ENV HUDI_HADOOP_BUNDLE=/var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/hood
 ENV HUDI_HIVE_BUNDLE=/var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/hoodie-hive-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_BENCH_BUNDLE=/var/hoodie/ws/docker/hoodie/hadoop/hive_base/target/hoodie-bench-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 497d9d4..8a13fb4 100644
--- a/docker/hoodie/hadoop/hive_base/pom.xml
+++ b/docker/hoodie/hadoop/hive_base/pom.xml
@@ -61,6 +61,9 @@
                 <copy
                   file="${project.basedir}/../../../../packaging/hudi-utilities-bundle/target/hudi-utilities-bundle-${project.version}.jar"
                   tofile="target/hoodie-utilities.jar"/>
+                <copy
+                  file="${project.basedir}/../../../../packaging/hudi-bench-bundle/target/hudi-bench-bundle-${project.version}.jar"
+                  tofile="target/hoodie-bench-bundle.jar"/>
               </tasks>
             </configuration>
             <goals>
diff --git a/hudi-bench/pom.xml b/hudi-bench/pom.xml
new file mode 100644
index 0000000..84b60b0
--- /dev/null
+++ b/hudi-bench/pom.xml
@@ -0,0 +1,357 @@
+<?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:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns="http://maven.apache.org/POM/4.0.0"
+  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.5.1-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>hudi-bench</artifactId>
+  <packaging>jar</packaging>
+
+  <properties>
+    <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-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.8</source>
+          <target>1.8</target>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>3.1.2</version>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </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>http://packages.confluent.io/maven/</url>
+    </repository>
+  </repositories>
+
+  <dependencies>
+
+    <!-- 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>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
+      <version>${jetty.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- the following order of dependencies are crucial -->
+
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_2.11</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>
+
+    <!-- Need this for SparkSession sparkSql queries -->
+    <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</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.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>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <classifier>tests</classifier>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </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>com.fasterxml.jackson.dataformat</groupId>
+      <artifactId>jackson-dataformat-yaml</artifactId>
+      <version>2.7.4</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro-mapred</artifactId>
+      <version>1.7.7</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-avro</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-utilities</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>
+      </exclusions>
+    </dependency>
+
+    <!-- Used for SQL templating -->
+    <dependency>
+      <groupId>org.antlr</groupId>
+      <artifactId>stringtemplate</artifactId>
+      <version>4.0.2</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <version>1.10.19</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-module-junit4</artifactId>
+      <version>1.7.3</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-api-mockito</artifactId>
+      <version>1.7.3</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>javassist</groupId>
+      <artifactId>javassist</artifactId>
+      <version>3.12.1.GA</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-utilities</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>
+      <classifier>tests</classifier>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>${hive.groupid}</groupId>
+      <artifactId>hive-exec</artifactId>
+      <version>${hive.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>servlet-api</artifactId>
+        </exclusion>
+      </exclusions>
+      <scope>test</scope>
+    </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>
+
+  </dependencies>
+
+
+</project>
\ No newline at end of file
diff --git a/hudi-bench/prepare_integration_suite.sh b/hudi-bench/prepare_integration_suite.sh
new file mode 100644
index 0000000..69ecdd9
--- /dev/null
+++ b/hudi-bench/prepare_integration_suite.sh
@@ -0,0 +1,112 @@
+#!/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-version"
+    echo "   -s, spark version"
+    echo "   -p, parquet version"
+    echo "   -a, avro version"
+    echo "   -s, hive version"
+    exit 1
+}
+
+get_spark_command() {
+echo "spark-submit --packages com.databricks:spark-avro_2.11:4.0.0 \
+--master $0 \
+--deploy-mode $1 \
+--properties-file $2 \
+--class org.apache.hudi.bench.job.HoodieTestSuiteJob \
+`ls target/hudi-bench-*-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:s:" 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"
+    ;;
+    s) hive="$OPTARG"
+    printf "Argument hive is %s\n" "$hive"
+    ;;
+    \?) 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
+  echo $base_command
+}
+
+versions=$(get_versions $hadoop $hive)
+
+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
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/DFSDeltaWriterAdapter.java b/hudi-bench/src/main/java/org/apache/hudi/bench/DFSDeltaWriterAdapter.java
new file mode 100644
index 0000000..e3f1f7b
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/DFSDeltaWriterAdapter.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bench;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.bench.writer.FileDeltaInputWriter;
+import org.apache.hudi.bench.writer.WriteStats;
+
+/**
+ * {@link org.apache.hadoop.hdfs.DistributedFileSystem} (or {@link org.apache.hadoop.fs.LocalFileSystem}) based delta
+ * generator.
+ */
+public class DFSDeltaWriterAdapter implements DeltaWriterAdapter<GenericRecord> {
+
+  private FileDeltaInputWriter deltaInputGenerator;
+  private List<WriteStats> metrics = new ArrayList<>();
+
+  public DFSDeltaWriterAdapter(FileDeltaInputWriter<GenericRecord> deltaInputGenerator) {
+    this.deltaInputGenerator = deltaInputGenerator;
+  }
+
+  @Override
+  public List<WriteStats> write(Iterator<GenericRecord> input) throws IOException {
+    deltaInputGenerator.open();
+    while (input.hasNext()) {
+      if (this.deltaInputGenerator.canWrite()) {
+        this.deltaInputGenerator.writeData(input.next());
+      } else if (input.hasNext()) {
+        rollOver();
+      }
+    }
+    close();
+    return this.metrics;
+  }
+
+  @VisibleForTesting
+  public void rollOver() throws IOException {
+    close();
+    this.deltaInputGenerator = this.deltaInputGenerator.getNewWriter();
+    this.deltaInputGenerator.open();
+  }
+
+  private void close() throws IOException {
+    this.deltaInputGenerator.close();
+    this.metrics.add(this.deltaInputGenerator.getWriteStats());
+  }
+}
diff --git a/hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/DFSSparkAvroDeltaWriter.java
similarity index 52%
copy from hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java
copy to hudi-bench/src/main/java/org/apache/hudi/bench/DFSSparkAvroDeltaWriter.java
index 50cf327..8c7c8b5 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/DFSSparkAvroDeltaWriter.java
@@ -16,26 +16,29 @@
  * limitations under the License.
  */
 
-package org.apache.hudi;
+package org.apache.hudi.bench;
 
+import java.io.IOException;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.bench.DeltaWriterAdapter.SparkBasedDeltaWriter;
+import org.apache.hudi.bench.writer.DeltaInputWriter;
+import org.apache.hudi.bench.writer.WriteStats;
+import org.apache.spark.api.java.JavaRDD;
 
 /**
- * Simple Key generator for unpartitioned Hive Tables
+ * NEED TO IMPLEMENT A CUSTOM SPARK PARTITIONER TO ENSURE WE WRITE LARGE ENOUGH AVRO FILES
  */
-public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
+public class DFSSparkAvroDeltaWriter implements SparkBasedDeltaWriter<JavaRDD<GenericRecord>> {
 
-  private static final String EMPTY_PARTITION = "";
+  private DeltaInputWriter<JavaRDD<GenericRecord>> deltaInputWriter;
 
-  public NonpartitionedKeyGenerator(TypedProperties props) {
-    super(props);
+  public DFSSparkAvroDeltaWriter(DeltaInputWriter<JavaRDD<GenericRecord>> deltaInputWriter) {
+    this.deltaInputWriter = deltaInputWriter;
   }
 
   @Override
-  public HoodieKey getKey(GenericRecord record) {
-    String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField);
-    return new HoodieKey(recordKey, EMPTY_PARTITION);
+  public JavaRDD<WriteStats> write(JavaRDD<GenericRecord> input) throws IOException {
+    this.deltaInputWriter.writeData(input);
+    return null;
   }
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/DeltaInputFormat.java
similarity index 57%
copy from hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
copy to hudi-bench/src/main/java/org/apache/hudi/bench/DeltaInputFormat.java
index c35663e..46d1425 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/DeltaInputFormat.java
@@ -16,26 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.hudi;
-
-import java.io.Serializable;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.util.TypedProperties;
+package org.apache.hudi.bench;
 
 /**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record
+ * Supported delta input data formats
  */
-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 DeltaInputFormat {
+  AVRO, PARQUET
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/DeltaOutputType.java
similarity index 57%
copy from hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
copy to hudi-bench/src/main/java/org/apache/hudi/bench/DeltaOutputType.java
index c35663e..77a1b0a 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/DeltaOutputType.java
@@ -16,26 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.hudi;
-
-import java.io.Serializable;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.util.TypedProperties;
+package org.apache.hudi.bench;
 
 /**
- * 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 DeltaOutputType {
+  KAFKA, DFS
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/DeltaWriterAdapter.java
similarity index 57%
copy from hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
copy to hudi-bench/src/main/java/org/apache/hudi/bench/DeltaWriterAdapter.java
index c35663e..da001db 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/DeltaWriterAdapter.java
@@ -16,26 +16,20 @@
  * limitations under the License.
  */
 
-package org.apache.hudi;
+package org.apache.hudi.bench;
 
-import java.io.Serializable;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.util.TypedProperties;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.hudi.bench.writer.WriteStats;
+import org.apache.spark.api.java.JavaRDD;
 
-/**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record
- */
-public abstract class KeyGenerator implements Serializable {
+public interface DeltaWriterAdapter<I> {
 
-  protected transient TypedProperties config;
+  List<WriteStats> write(Iterator<I> input) throws IOException;
 
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
-  }
+  interface SparkBasedDeltaWriter<J> {
 
-  /**
-   * Generate a Hoodie Key out of provided generic record.
-   */
-  public abstract HoodieKey getKey(GenericRecord record);
+    JavaRDD<WriteStats> write(J input) throws IOException;
+  }
 }
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/DeltaWriterFactory.java b/hudi-bench/src/main/java/org/apache/hudi/bench/DeltaWriterFactory.java
new file mode 100644
index 0000000..aeefa48
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/DeltaWriterFactory.java
@@ -0,0 +1,59 @@
+/*
+ * 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.bench;
+
+import java.io.IOException;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.bench.configuration.DFSDeltaConfig;
+import org.apache.hudi.bench.configuration.DeltaConfig;
+import org.apache.hudi.bench.writer.AvroDeltaInputWriter;
+import org.apache.hudi.bench.writer.FileDeltaInputWriter;
+import org.apache.hudi.common.util.StringUtils;
+
+/**
+ * A factory to help instantiate different {@link DeltaWriterAdapter}s depending on the {@link DeltaOutputType} and
+ * {@link DeltaInputFormat}
+ */
+public class DeltaWriterFactory {
+
+  private DeltaWriterFactory() {
+  }
+
+  public static DeltaWriterAdapter getDeltaWriterAdapter(DeltaConfig config, Integer batchId) throws IOException {
+    switch (config.getDeltaOutputType()) {
+      case DFS:
+        switch (config.getDeltaInputFormat()) {
+          case AVRO:
+            DFSDeltaConfig dfsDeltaConfig = (DFSDeltaConfig) config;
+            dfsDeltaConfig.setBatchId(batchId);
+            FileDeltaInputWriter<GenericRecord> fileDeltaInputGenerator = new AvroDeltaInputWriter(
+                dfsDeltaConfig.getConfiguration(),
+                StringUtils
+                    .join(new String[]{dfsDeltaConfig.getDeltaBasePath(), dfsDeltaConfig.getBatchId().toString()},
+                        "/"), dfsDeltaConfig.getSchemaStr(), dfsDeltaConfig.getMaxFileSize());
+            DFSDeltaWriterAdapter workloadSink = new DFSDeltaWriterAdapter(fileDeltaInputGenerator);
+            return workloadSink;
+          default:
+            throw new IllegalArgumentException("Invalid delta input format " + config.getDeltaInputFormat());
+        }
+      default:
+        throw new IllegalArgumentException("Invalid delta input type " + config.getDeltaOutputType());
+    }
+  }
+}
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/configuration/DFSDeltaConfig.java b/hudi-bench/src/main/java/org/apache/hudi/bench/configuration/DFSDeltaConfig.java
new file mode 100644
index 0000000..3781f3c
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.configuration;
+
+import org.apache.hudi.bench.DeltaInputFormat;
+import org.apache.hudi.bench.DeltaOutputType;
+import org.apache.hudi.common.SerializableConfiguration;
+
+/**
+ * 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(DeltaOutputType deltaOutputType, DeltaInputFormat deltaInputFormat,
+      SerializableConfiguration configuration,
+      String deltaBasePath, String targetBasePath, String schemaStr, Long maxFileSize) {
+    super(deltaOutputType, deltaInputFormat, 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-bench/src/main/java/org/apache/hudi/bench/configuration/DeltaConfig.java b/hudi-bench/src/main/java/org/apache/hudi/bench/configuration/DeltaConfig.java
new file mode 100644
index 0000000..11aa072
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/configuration/DeltaConfig.java
@@ -0,0 +1,258 @@
+/*
+ * 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.bench.configuration;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+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.bench.DeltaInputFormat;
+import org.apache.hudi.bench.DeltaOutputType;
+import org.apache.hudi.common.SerializableConfiguration;
+import org.apache.hudi.common.util.collection.Pair;
+
+/**
+ * Configuration to hold the delta output type and delta input format
+ */
+public class DeltaConfig implements Serializable {
+
+  private final DeltaOutputType deltaOutputType;
+  private final DeltaInputFormat deltaInputFormat;
+  private final SerializableConfiguration configuration;
+
+  public DeltaConfig(DeltaOutputType deltaOutputType, DeltaInputFormat deltaInputFormat,
+      SerializableConfiguration configuration) {
+    this.deltaOutputType = deltaOutputType;
+    this.deltaInputFormat = deltaInputFormat;
+    this.configuration = configuration;
+  }
+
+  public DeltaOutputType getDeltaOutputType() {
+    return deltaOutputType;
+  }
+
+  public DeltaInputFormat getDeltaInputFormat() {
+    return deltaInputFormat;
+  }
+
+  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;
+
+    @VisibleForTesting
+    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-bench/src/main/java/org/apache/hudi/bench/converter/UpdateConverter.java b/hudi-bench/src/main/java/org/apache/hudi/bench/converter/UpdateConverter.java
new file mode 100644
index 0000000..7cf48c3
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/converter/UpdateConverter.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.bench.converter;
+
+import java.util.List;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.bench.generator.LazyRecordGeneratorIterator;
+import org.apache.hudi.bench.generator.UpdateGeneratorIterator;
+import org.apache.hudi.utilities.converter.Converter;
+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-bench/src/main/java/org/apache/hudi/bench/dag/DagUtils.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/DagUtils.java
new file mode 100644
index 0000000..93a04b3
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/DagUtils.java
@@ -0,0 +1,218 @@
+/*
+ * 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.bench.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.bench.configuration.DeltaConfig;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.nodes.DagNode;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+
+/**
+ * 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(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(Config.TYPE).asText();
+    final DagNode retNode = convertJsonToDagNode(node, type);
+    Arrays.asList(node.get(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 = 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(Config.CONFIG_NAME).fields();
+    while (itr.hasNext()) {
+      Entry<String, JsonNode> entry = itr.next();
+      switch (entry.getKey()) {
+        case Config.HIVE_QUERIES:
+          configsMap.put(Config.HIVE_QUERIES, getHiveQueries(entry));
+          break;
+        case Config.HIVE_PROPERTIES:
+          configsMap.put(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(Config.CONFIG_NAME, configNode);
+    ((ObjectNode) jsonNode).put(Config.TYPE, type);
+    ((ObjectNode) jsonNode).put(Config.DEPENDENCIES, getDependencyNames(node));
+    return jsonNode;
+  }
+
+  private static String getDependencyNames(DagNode node) {
+    return node.getParentNodes().stream()
+        .map(e -> ((DagNode) e).getConfig().getOtherConfigs().getOrDefault(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-bench/src/main/java/org/apache/hudi/bench/dag/ExecutionContext.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/ExecutionContext.java
new file mode 100644
index 0000000..03b585f
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/ExecutionContext.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.bench.dag;
+
+import java.io.Serializable;
+import org.apache.hudi.bench.generator.DeltaGenerator;
+import org.apache.hudi.bench.writer.DeltaWriter;
+import org.apache.spark.api.java.JavaSparkContext;
+
+/**
+ * This wraps the context needed for an execution of
+ * a {@link org.apache.hudi.bench.dag.nodes.DagNode#execute(ExecutionContext)}
+ */
+public class ExecutionContext implements Serializable {
+
+  private DeltaWriter deltaWriter;
+  private DeltaGenerator deltaGenerator;
+  private transient JavaSparkContext jsc;
+
+  public ExecutionContext(JavaSparkContext jsc, DeltaWriter deltaWriter, DeltaGenerator deltaGenerator) {
+    this.deltaWriter = deltaWriter;
+    this.deltaGenerator = deltaGenerator;
+    this.jsc = jsc;
+  }
+
+  public DeltaWriter getDeltaWriter() {
+    return deltaWriter;
+  }
+
+  public DeltaGenerator getDeltaGenerator() {
+    return deltaGenerator;
+  }
+
+  public JavaSparkContext getJsc() {
+    return jsc;
+  }
+}
diff --git a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/WorkflowDag.java
similarity index 57%
copy from hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
copy to hudi-bench/src/main/java/org/apache/hudi/bench/dag/WorkflowDag.java
index c35663e..b88bd74 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/WorkflowDag.java
@@ -16,26 +16,24 @@
  * limitations under the License.
  */
 
-package org.apache.hudi;
+package org.apache.hudi.bench.dag;
 
-import java.io.Serializable;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.util.TypedProperties;
+import java.util.List;
+import org.apache.hudi.bench.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;
 
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
+  public WorkflowDag(List<DagNode<O>> nodeList) {
+    this.nodeList = nodeList;
+  }
+
+  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-bench/src/main/java/org/apache/hudi/bench/dag/WorkflowDagGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/WorkflowDagGenerator.java
new file mode 100644
index 0000000..a5602ce
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/WorkflowDagGenerator.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.bench.dag;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.nodes.DagNode;
+import org.apache.hudi.bench.dag.nodes.HiveQueryNode;
+import org.apache.hudi.bench.dag.nodes.InsertNode;
+import org.apache.hudi.bench.dag.nodes.UpsertNode;
+import org.apache.hudi.common.util.collection.Pair;
+
+/**
+ * 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 WorkflowDagGenerator {
+
+  public WorkflowDag build() {
+
+    DagNode root = new InsertNode(Config.newBuilder()
+        .withNumRecordsToInsert(100)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(2)
+        .withRecordSize(1000).build());
+
+    DagNode child1 = new InsertNode(Config.newBuilder()
+        .withNumRecordsToInsert(100)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(2)
+        .withRecordSize(1000).build());
+
+    root.addChildNode(child1);
+
+    DagNode child1OfChild1 = new UpsertNode(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.hive_trips group "
+        + "by rider having count(*) < 1", 0));
+    DagNode child2OfChild1 = new HiveQueryNode(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/NonpartitionedKeyGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/BulkInsertNode.java
similarity index 57%
copy from hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java
copy to hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/BulkInsertNode.java
index 50cf327..e855075 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/BulkInsertNode.java
@@ -16,26 +16,25 @@
  * limitations under the License.
  */
 
-package org.apache.hudi;
+package org.apache.hudi.bench.dag.nodes;
 
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.WriteStatus;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.writer.DeltaWriter;
+import org.apache.hudi.common.util.Option;
+import org.apache.spark.api.java.JavaRDD;
 
-/**
- * Simple Key generator for unpartitioned Hive Tables
- */
-public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
-
-  private static final String EMPTY_PARTITION = "";
+public class BulkInsertNode extends InsertNode {
 
-  public NonpartitionedKeyGenerator(TypedProperties props) {
-    super(props);
+  public BulkInsertNode(Config config) {
+    super(config);
   }
 
   @Override
-  public HoodieKey getKey(GenericRecord record) {
-    String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField);
-    return new HoodieKey(recordKey, EMPTY_PARTITION);
+  protected JavaRDD<WriteStatus> ingest(DeltaWriter deltaWriter, Option<String> commitTime)
+      throws Exception {
+    log.info("Execute bulk ingest node " + this.getName());
+    return deltaWriter.bulkInsert(commitTime);
   }
+
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/CleanNode.java
similarity index 57%
copy from hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java
copy to hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/CleanNode.java
index 50cf327..6e43259 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/CleanNode.java
@@ -16,26 +16,19 @@
  * limitations under the License.
  */
 
-package org.apache.hudi;
+package org.apache.hudi.bench.dag.nodes;
 
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.bench.dag.ExecutionContext;
 
-/**
- * Simple Key generator for unpartitioned Hive Tables
- */
-public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
-
-  private static final String EMPTY_PARTITION = "";
+public class CleanNode extends DagNode<Boolean> {
 
-  public NonpartitionedKeyGenerator(TypedProperties props) {
-    super(props);
+  public CleanNode() {
   }
 
   @Override
-  public HoodieKey getKey(GenericRecord record) {
-    String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField);
-    return new HoodieKey(recordKey, EMPTY_PARTITION);
+  public void execute(ExecutionContext executionContext) throws Exception {
+    log.info("Executing clean node " + this.getName());
+    executionContext.getDeltaWriter().getWriteClient().clean();
   }
+
 }
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/CompactNode.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/CompactNode.java
new file mode 100644
index 0000000..2c65252
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.dag.nodes;
+
+import org.apache.hudi.WriteStatus;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.ExecutionContext;
+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.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.getDeltaWriter().getConfiguration(),
+        executionContext.getDeltaWriter().getCfg().targetBasePath);
+    Option<HoodieInstant> lastInstant = metaClient.getActiveTimeline()
+        .getCommitsAndCompactionTimeline().filterPendingCompactionTimeline().lastInstant();
+    if (lastInstant.isPresent()) {
+      log.info("Compacting instant => " + lastInstant.get());
+      this.result = executionContext.getDeltaWriter().compact(Option.of(lastInstant.get().getTimestamp()));
+    }
+  }
+
+}
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/DagNode.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/DagNode.java
new file mode 100644
index 0000000..3f4083a
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.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.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.ExecutionContext;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Represents a Node in the DAG of operations for a workflow.
+ */
+public abstract class DagNode<O> implements Comparable<DagNode<O>> {
+
+  protected static Logger log = LogManager.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-bench/src/main/java/org/apache/hudi/bench/dag/nodes/HiveQueryNode.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/HiveQueryNode.java
new file mode 100644
index 0000000..8c5cac5
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/HiveQueryNode.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.bench.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.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.ExecutionContext;
+import org.apache.hudi.bench.helpers.HiveServiceProvider;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.hive.HiveSyncConfig;
+
+public class HiveQueryNode extends DagNode<Boolean> {
+
+  private HiveServiceProvider hiveServiceProvider;
+
+  public HiveQueryNode(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.getDeltaWriter().getConfiguration());
+    // this.hiveServiceProvider.syncToLocalHiveIfNeeded(writer);
+    HiveSyncConfig hiveSyncConfig = DataSourceUtils
+        .buildHiveSyncConfig(executionContext.getDeltaWriter().getDeltaStreamerWrapper()
+                .getDeltaSyncService().getDeltaSync().getProps(),
+            executionContext.getDeltaWriter().getDeltaStreamerWrapper()
+                .getDeltaSyncService().getDeltaSync().getCfg().targetBasePath);
+    this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getDeltaWriter());
+    Connection con = DriverManager.getConnection(hiveSyncConfig.jdbcUrl, hiveSyncConfig.hiveUser,
+        hiveSyncConfig.hivePass);
+    Statement stmt = con.createStatement();
+    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.getRow() == 0) {
+        assert 0 == queryAndResult.getRight();
+      } else {
+        assert res.getInt(0) == queryAndResult.getRight();
+      }
+      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-bench/src/main/java/org/apache/hudi/bench/dag/nodes/HiveSyncNode.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/HiveSyncNode.java
new file mode 100644
index 0000000..7ad06c6
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.dag.nodes;
+
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.ExecutionContext;
+import org.apache.hudi.bench.helpers.HiveServiceProvider;
+
+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.getDeltaWriter().getConfiguration());
+    this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getDeltaWriter());
+    executionContext.getDeltaWriter().getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync().syncHive();
+    this.hiveServiceProvider.stopLocalHiveServiceIfNeeded();
+  }
+
+  public HiveServiceProvider getHiveServiceProvider() {
+    return hiveServiceProvider;
+  }
+}
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/InsertNode.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/InsertNode.java
new file mode 100644
index 0000000..73a1e22
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/InsertNode.java
@@ -0,0 +1,65 @@
+/*
+ * 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.bench.dag.nodes;
+
+import org.apache.hudi.WriteStatus;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.ExecutionContext;
+import org.apache.hudi.bench.generator.DeltaGenerator;
+import org.apache.hudi.bench.writer.DeltaWriter;
+import org.apache.hudi.common.util.Option;
+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(String.format("----------------- inserting input data %s ------------------", this.getName()));
+      Option<String> commitTime = executionContext.getDeltaWriter().startCommit();
+      JavaRDD<WriteStatus> writeStatus = ingest(executionContext.getDeltaWriter(), commitTime);
+      executionContext.getDeltaWriter().commit(writeStatus, commitTime);
+      this.result = writeStatus;
+    }
+    validate();
+  }
+
+  protected void generate(DeltaGenerator deltaGenerator) throws Exception {
+    if (!config.isDisableGenerate()) {
+      log.info(String.format("----------------- generating input data for node %s ------------------", this.getName()));
+      deltaGenerator.writeRecords(deltaGenerator.generateInserts(config)).count();
+    }
+  }
+
+  protected JavaRDD<WriteStatus> ingest(DeltaWriter deltaWriter,
+      Option<String> commitTime) throws Exception {
+    return deltaWriter.insert(commitTime);
+  }
+
+  protected boolean validate() {
+    return this.result.count() == config.getNumRecordsInsert();
+  }
+
+}
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/RollbackNode.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/RollbackNode.java
new file mode 100644
index 0000000..1128368
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.dag.nodes;
+
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.ExecutionContext;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+
+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.getDeltaWriter().getConfiguration(),
+        executionContext.getDeltaWriter().getCfg().targetBasePath);
+    Option<HoodieInstant> lastInstant = metaClient.getActiveTimeline().getCommitsTimeline().lastInstant();
+    if (lastInstant.isPresent()) {
+      log.info("Rolling back last instant => " + lastInstant.get());
+      executionContext.getDeltaWriter().getWriteClient().rollback(lastInstant.get().getTimestamp());
+      this.result = lastInstant;
+    }
+  }
+
+}
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/ScheduleCompactNode.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/ScheduleCompactNode.java
new file mode 100644
index 0000000..d983af2
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/ScheduleCompactNode.java
@@ -0,0 +1,55 @@
+/*
+ * 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.bench.dag.nodes;
+
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.ExecutionContext;
+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;
+
+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.getDeltaWriter().getConfiguration(),
+        executionContext.getDeltaWriter().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.getDeltaWriter().getWriteClient()
+          .scheduleCompaction(Option.of(metadata
+              .getExtraMetadata()));
+      log.info("Scheduling compaction instant => " + scheduledInstant.get());
+      this.result = scheduledInstant;
+    }
+  }
+
+}
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/SparkSQLQueryNode.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/SparkSQLQueryNode.java
new file mode 100644
index 0000000..d6d931e
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.dag.nodes;
+
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.ExecutionContext;
+import org.apache.hudi.bench.helpers.HiveServiceProvider;
+import org.apache.hudi.common.util.collection.Pair;
+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.getDeltaWriter().getConfiguration());
+    this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getDeltaWriter());
+    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-bench/src/main/java/org/apache/hudi/bench/dag/nodes/UpsertNode.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/UpsertNode.java
new file mode 100644
index 0000000..60b013f
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/UpsertNode.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.bench.dag.nodes;
+
+import org.apache.hudi.WriteStatus;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.generator.DeltaGenerator;
+import org.apache.hudi.bench.writer.DeltaWriter;
+import org.apache.hudi.common.util.Option;
+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(String.format("----------------- generating input data %s ------------------", this.getName()));
+      deltaGenerator.writeRecords(deltaGenerator.generateUpdates(config)).count();
+    }
+  }
+
+  @Override
+  protected JavaRDD<WriteStatus> ingest(DeltaWriter deltaWriter, Option<String> commitTime)
+      throws Exception {
+    if (!config.isDisableIngest()) {
+      log.info(String.format("----------------- upserting input data %s ------------------", this.getName()));
+      this.result = deltaWriter.upsert(commitTime);
+    }
+    return this.result;
+  }
+
+  @Override
+  protected boolean validate() {
+    return this.result.count() == this.config.getNumRecordsInsert() + this.config.getNumRecordsUpsert();
+  }
+}
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/ValidateNode.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/nodes/ValidateNode.java
new file mode 100644
index 0000000..3f28abc
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.dag.nodes;
+
+import java.util.List;
+import java.util.function.Function;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.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-bench/src/main/java/org/apache/hudi/bench/dag/scheduler/DagScheduler.java b/hudi-bench/src/main/java/org/apache/hudi/bench/dag/scheduler/DagScheduler.java
new file mode 100644
index 0000000..1dde398
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.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.bench.dag.ExecutionContext;
+import org.apache.hudi.bench.dag.WorkflowDag;
+import org.apache.hudi.bench.dag.nodes.DagNode;
+import org.apache.hudi.bench.generator.DeltaGenerator;
+import org.apache.hudi.bench.writer.DeltaWriter;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class DagScheduler {
+
+  private static Logger log = LogManager.getLogger(DagScheduler.class);
+  private WorkflowDag workflowDag;
+  private ExecutionContext executionContext;
+
+  public DagScheduler(WorkflowDag workflowDag, DeltaWriter deltaWriter, DeltaGenerator deltaGenerator) {
+    this.workflowDag = workflowDag;
+    this.executionContext = new ExecutionContext(null, deltaWriter, 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-bench/src/main/java/org/apache/hudi/bench/generator/DeltaGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/generator/DeltaGenerator.java
new file mode 100644
index 0000000..335fecf
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.generator;
+
+import com.google.common.annotations.VisibleForTesting;
+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.KeyGenerator;
+import org.apache.hudi.bench.DeltaOutputType;
+import org.apache.hudi.bench.DeltaWriterAdapter;
+import org.apache.hudi.bench.DeltaWriterFactory;
+import org.apache.hudi.bench.configuration.DFSDeltaConfig;
+import org.apache.hudi.bench.configuration.DeltaConfig;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.converter.UpdateConverter;
+import org.apache.hudi.bench.reader.DFSAvroDeltaInputReader;
+import org.apache.hudi.bench.reader.DFSHoodieDatasetInputReader;
+import org.apache.hudi.bench.reader.DeltaInputReader;
+import org.apache.hudi.bench.writer.WriteStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.utilities.converter.Converter;
+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;
+import org.apache.spark.storage.StorageLevel;
+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 = LogManager.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.getRecordKeyFields();
+    this.partitionPathFieldNames = keyGenerator.getPartitionPathFields();
+  }
+
+  public JavaRDD<WriteStats> writeRecords(JavaRDD<GenericRecord> records) {
+    // The following creates a new anonymous function for iterator and hence results in serialization issues
+    JavaRDD<WriteStats> 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.getDeltaOutputType() == DeltaOutputType.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");
+    }
+  }
+
+  @VisibleForTesting
+  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();
+  }
+
+  @VisibleForTesting
+  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;
+  }
+
+  @VisibleForTesting
+  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-bench/src/main/java/org/apache/hudi/bench/generator/FlexibleSchemaRecordGenerationIterator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/generator/FlexibleSchemaRecordGenerationIterator.java
new file mode 100644
index 0000000..a982792
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.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-bench/src/main/java/org/apache/hudi/bench/generator/GenericRecordFullPayloadGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/generator/GenericRecordFullPayloadGenerator.java
new file mode 100644
index 0000000..2232cc5
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/generator/GenericRecordFullPayloadGenerator.java
@@ -0,0 +1,245 @@
+/*
+ * 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.bench.generator;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Type;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * 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 = LogManager.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;
+
+  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;
+      default:
+        throw new IllegalArgumentException(
+            "Cannot handle type: " + localSchema.getType());
+    }
+  }
+
+  @VisibleForTesting
+  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(Type type) {
+    switch (type) {
+      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;
+      default:
+        throw new RuntimeException("Unknown type " + type);
+    }
+  }
+
+  private int numEntriesToAdd(Schema elementSchema) {
+    // Find the size of the primitive data type in bytes
+    int primitiveDataTypeSize = getSize(elementSchema.getType());
+    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-bench/src/main/java/org/apache/hudi/bench/generator/GenericRecordFullPayloadSizeEstimator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/generator/GenericRecordFullPayloadSizeEstimator.java
new file mode 100644
index 0000000..1d515e9
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/generator/GenericRecordFullPayloadSizeEstimator.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.bench.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);
+      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-bench/src/main/java/org/apache/hudi/bench/generator/GenericRecordPartialPayloadGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/generator/GenericRecordPartialPayloadGenerator.java
new file mode 100644
index 0000000..04174cd
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/generator/GenericRecordPartialPayloadGenerator.java
@@ -0,0 +1,84 @@
+/*
+ * 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.bench.generator;
+
+import com.google.common.annotations.VisibleForTesting;
+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()));
+        }
+      }
+    }
+  }
+
+  @VisibleForTesting
+  @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/NonpartitionedKeyGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/generator/LazyRecordGeneratorIterator.java
similarity index 60%
copy from hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java
copy to hudi-bench/src/main/java/org/apache/hudi/bench/generator/LazyRecordGeneratorIterator.java
index 50cf327..902aaee 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/generator/LazyRecordGeneratorIterator.java
@@ -16,26 +16,33 @@
  * limitations under the License.
  */
 
-package org.apache.hudi;
+package org.apache.hudi.bench.generator;
 
+import java.util.Iterator;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.func.LazyIterableIterator;
 
 /**
- * Simple Key generator for unpartitioned Hive Tables
+ * A lazy record generator to generate {@link GenericRecord}s lazily and not hold a list of records in memory
  */
-public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
+public class LazyRecordGeneratorIterator extends
+    LazyIterableIterator<GenericRecord, GenericRecord> {
 
-  private static final String EMPTY_PARTITION = "";
+  public LazyRecordGeneratorIterator(Iterator<GenericRecord> inputItr) {
+    super(inputItr);
+  }
+
+  @Override
+  protected void start() {
+  }
 
-  public NonpartitionedKeyGenerator(TypedProperties props) {
-    super(props);
+  @Override
+  protected GenericRecord computeNext() {
+    return inputItr.next();
   }
 
   @Override
-  public HoodieKey getKey(GenericRecord record) {
-    String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField);
-    return new HoodieKey(recordKey, EMPTY_PARTITION);
+  protected void end() {
+
   }
 }
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/generator/UpdateGeneratorIterator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/generator/UpdateGeneratorIterator.java
new file mode 100644
index 0000000..235b090
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.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-bench/src/main/java/org/apache/hudi/bench/helpers/DFSTestSuitePathSelector.java b/hudi-bench/src/main/java/org/apache/hudi/bench/helpers/DFSTestSuitePathSelector.java
new file mode 100644
index 0000000..3167407
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.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.util.Option;
+import org.apache.hudi.common.util.TypedProperties;
+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-bench/src/main/java/org/apache/hudi/bench/helpers/HiveServiceProvider.java b/hudi-bench/src/main/java/org/apache/hudi/bench/helpers/HiveServiceProvider.java
new file mode 100644
index 0000000..5ae2ed1
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/helpers/HiveServiceProvider.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bench.helpers;
+
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hive.service.server.HiveServer2;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.writer.DeltaWriter;
+import org.apache.hudi.hive.util.HiveTestService;
+
+public class HiveServiceProvider {
+
+  private HiveTestService hiveService;
+  private HiveServer2 hiveServer;
+  private Config config;
+
+  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(DeltaWriter 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()) {
+      hiveServer.stop();
+      hiveService.stop();
+    }
+  }
+
+  public HiveServer2 getLocalHiveServer() {
+    return hiveServer;
+  }
+}
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/job/HoodieDeltaStreamerWrapper.java b/hudi-bench/src/main/java/org/apache/hudi/bench/job/HoodieDeltaStreamerWrapper.java
new file mode 100644
index 0000000..e0f77df
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/job/HoodieDeltaStreamerWrapper.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.bench.job;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hudi.WriteStatus;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+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(Option<String> instantTime) throws
+      Exception {
+    return deltaSyncService.getDeltaSync().syncOnce().getRight();
+  }
+
+  public JavaRDD<WriteStatus> insert(Option<String> instantTime) throws Exception {
+    return upsert(instantTime);
+  }
+
+  public JavaRDD<WriteStatus> bulkInsert(Option<String> instantTime) throws
+      Exception {
+    return upsert(instantTime);
+  }
+
+  public JavaRDD<WriteStatus> compact(Option<String> instantTime) throws Exception {
+    return upsert(instantTime);
+  }
+
+  public Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> fetchSource() throws Exception {
+    return deltaSyncService.getDeltaSync().readFromSource(deltaSyncService.getDeltaSync().getCommitTimelineOpt());
+  }
+
+}
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/job/HoodieTestSuiteJob.java b/hudi-bench/src/main/java/org/apache/hudi/bench/job/HoodieTestSuiteJob.java
new file mode 100644
index 0000000..b9e52d3
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/job/HoodieTestSuiteJob.java
@@ -0,0 +1,184 @@
+/*
+ * 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.bench.job;
+
+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.KeyGenerator;
+import org.apache.hudi.bench.DeltaInputFormat;
+import org.apache.hudi.bench.DeltaOutputType;
+import org.apache.hudi.bench.configuration.DFSDeltaConfig;
+import org.apache.hudi.bench.dag.DagUtils;
+import org.apache.hudi.bench.dag.WorkflowDag;
+import org.apache.hudi.bench.dag.WorkflowDagGenerator;
+import org.apache.hudi.bench.dag.scheduler.DagScheduler;
+import org.apache.hudi.bench.generator.DeltaGenerator;
+import org.apache.hudi.bench.writer.DeltaWriter;
+import org.apache.hudi.common.SerializableConfiguration;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+
+/**
+ * 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 = LogManager.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,
+          cfg.storageType, 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() throws Exception {
+    try {
+      WorkflowDag workflowDag = this.cfg.workloadYamlPath == null ? ((WorkflowDagGenerator) ReflectionUtils
+          .loadClass((this.cfg).workloadDagGenerator)).build()
+          : DagUtils.convertYamlPathToDag(this.fs, this.cfg.workloadYamlPath);
+      String schemaStr = schemaProvider.getSourceSchema().toString();
+      final DeltaWriter writer = new DeltaWriter(jsc, props, cfg, schemaStr);
+      final DeltaGenerator deltaGenerator = new DeltaGenerator(
+          new DFSDeltaConfig(DeltaOutputType.valueOf(cfg.outputTypeName), DeltaInputFormat.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");
+    } 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.bench.workload.DeltaOutputType to readAvro data.")
+    public String outputTypeName = DeltaOutputType.DFS.name();
+
+    @Parameter(names = {"--delta-input-format"}, description = "Subclass of "
+        + "org.apache.hudi.bench.workload.DeltaOutputType to read avro data.")
+    public String inputFormatName = DeltaInputFormat.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-bench/src/main/java/org/apache/hudi/bench/reader/DFSAvroDeltaInputReader.java b/hudi-bench/src/main/java/org/apache/hudi/bench/reader/DFSAvroDeltaInputReader.java
new file mode 100644
index 0000000..7d06c77
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/reader/DFSAvroDeltaInputReader.java
@@ -0,0 +1,90 @@
+/*
+ * 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.bench.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.bench.DeltaInputFormat;
+import org.apache.hudi.bench.DeltaOutputType;
+import org.apache.hudi.bench.writer.AvroDeltaInputWriter;
+import org.apache.hudi.common.util.Option;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.SparkSession;
+
+/**
+ * A reader of {@link DeltaOutputType#DFS} and {@link DeltaInputFormat#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(AvroDeltaInputWriter.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-bench/src/main/java/org/apache/hudi/bench/reader/DFSDeltaInputReader.java b/hudi-bench/src/main/java/org/apache/hudi/bench/reader/DFSDeltaInputReader.java
new file mode 100644
index 0000000..9d3e5f2
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.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.util.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-bench/src/main/java/org/apache/hudi/bench/reader/DFSHoodieDatasetInputReader.java b/hudi-bench/src/main/java/org/apache/hudi/bench/reader/DFSHoodieDatasetInputReader.java
new file mode 100644
index 0000000..462acc7
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/reader/DFSHoodieDatasetInputReader.java
@@ -0,0 +1,286 @@
+/*
+ * 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.bench.reader;
+
+import static java.util.Map.Entry.comparingByValue;
+import static java.util.stream.Collectors.toMap;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+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.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.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.TableFileSystemView;
+import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.HoodieAvroUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieMemoryConfig;
+import org.apache.hudi.func.ParquetReaderIterator;
+import org.apache.hudi.hadoop.realtime.AbstractRealtimeRecordReader;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+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 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 = LogManager.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()) {
+      Preconditions.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.RealtimeView 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 " + numPartitions + ", NumFiles " + numFiles + " numRecordsToUpdate "
+        + numRecordsToUpdate + " percentageRecordsPerFile " + 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 = Iterators.size(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()) {
+      updates = updates.union(projectSchema(generateUpdates(adjustedPartitionToFileIdCountMap,
+          partitionToFileSlice, numFilesToUpdate, (int) (numRecordsToUpdate.get() - numRecordsToUpdatePerFile * numFiles
+              .get()))));
+    }
+    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 Iterators.limit(p._2, maxFilesToRead);
+    }).flatMap(p -> p).repartition(numFiles).map(fileSlice -> {
+      if (numRecordsToReadPerFile > 0) {
+        return Iterators.limit(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, Iterators
+        .size(p._2))).collectAsMap();
+    long totalExistingFilesCount = partitionToFileIdCountMap.values().stream().reduce((a, b) -> a + b).get();
+    Preconditions.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.getDataFile().isPresent()) {
+        newSlice.setDataFile(slice.getDataFile().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.getDataFile().isPresent()) {
+      Iterator<IndexedRecord> itr =
+          new ParquetReaderIterator<IndexedRecord>(AvroParquetReader.<IndexedRecord>builder(new
+              Path(fileSlice.getDataFile().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,
+          AbstractRealtimeRecordReader.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();
+    }
+  }
+
+  @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-bench/src/main/java/org/apache/hudi/bench/reader/DFSParquetDeltaInputReader.java b/hudi-bench/src/main/java/org/apache/hudi/bench/reader/DFSParquetDeltaInputReader.java
new file mode 100644
index 0000000..c0016a2
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/reader/DFSParquetDeltaInputReader.java
@@ -0,0 +1,90 @@
+/*
+ * 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.bench.reader;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hudi.bench.DeltaInputFormat;
+import org.apache.hudi.bench.DeltaOutputType;
+import org.apache.hudi.common.util.Option;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.SparkSession;
+
+/**
+ * A reader of {@link DeltaOutputType#DFS} and {@link DeltaInputFormat#PARQUET}
+ */
+public class DFSParquetDeltaInputReader extends DFSDeltaInputReader {
+
+  private static final String PARQUET_EXTENSION = ".parquet";
+  private final SparkSession sparkSession;
+  private final String basePath;
+  private final Option<String> structName;
+  private final Option<String> nameSpace;
+  protected PathFilter filter = (path) -> {
+    if (path.toUri().toString().contains(PARQUET_EXTENSION)) {
+      return true;
+    } else {
+      return false;
+    }
+  };
+
+  public DFSParquetDeltaInputReader(SparkSession sparkSession, String schemaStr, String basePath,
+      Option<String> structName, Option<String> nameSpace) {
+    this.sparkSession = sparkSession;
+    this.basePath = basePath;
+    this.structName = structName;
+    this.nameSpace = nameSpace;
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> read(long totalRecordsToRead) throws IOException {
+    List<String> parquetFiles = getFilePathsToRead(basePath, filter, totalRecordsToRead);
+    if (parquetFiles.size() > 0) {
+      return SparkBasedReader.readParquet(sparkSession, parquetFiles, structName, nameSpace);
+    } else {
+      throw new UnsupportedOperationException("Cannot read other format");
+    }
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> read(int numPartitions, long approxNumRecords) {
+    throw new UnsupportedOperationException("cannot generate updates");
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> read(int numPartitions, int numFiles, long approxNumRecords) {
+    throw new UnsupportedOperationException("cannot generate updates");
+  }
+
+  @Override
+  public JavaRDD<GenericRecord> read(int numPartitions, int numFiles, double percentageRecordsPerFile) {
+    throw new UnsupportedOperationException("cannot generate updates");
+  }
+
+  @Override
+  protected long analyzeSingleFile(String filePath) {
+    JavaRDD<GenericRecord> recordsFromOneFile = SparkBasedReader.readParquet(sparkSession, Arrays.asList(filePath),
+        structName, nameSpace);
+    return recordsFromOneFile.count();
+  }
+
+}
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/reader/DeltaInputReader.java b/hudi-bench/src/main/java/org/apache/hudi/bench/reader/DeltaInputReader.java
new file mode 100644
index 0000000..c322c88
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/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.bench.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-bench/src/main/java/org/apache/hudi/bench/reader/SparkBasedReader.java b/hudi-bench/src/main/java/org/apache/hudi/bench/reader/SparkBasedReader.java
new file mode 100644
index 0000000..6a28923
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/reader/SparkBasedReader.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bench.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.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 = "com.databricks.spark.avro";
+  public static final String SPARK_PARQUET_FORMAT = "com.databricks.spark.parquet";
+  private static final String AVRO_SCHEMA_OPTION_KEY = "avroSchema";
+  private static final String DEFAULT_STRUCT_NAME = "test.struct";
+  private static final String DEFAULT_NAMESPACE_NAME = "test.namespace";
+
+  // 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(DEFAULT_STRUCT_NAME), nameSpace.orElse(DEFAULT_NAMESPACE_NAME))
+        .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(DEFAULT_STRUCT_NAME), nameSpace.orElse(DEFAULT_NAMESPACE_NAME))
+        .toJavaRDD();
+  }
+
+}
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/writer/AvroDeltaInputWriter.java b/hudi-bench/src/main/java/org/apache/hudi/bench/writer/AvroDeltaInputWriter.java
new file mode 100644
index 0000000..d53c39c
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/writer/AvroDeltaInputWriter.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.bench.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.io.storage.HoodieWrapperFileSystem;
+import org.apache.hudi.io.storage.HoodieParquetWriter;
+import org.apache.log4j.Logger;
+
+/**
+ * Implementation of {@link FileDeltaInputWriter} that writes avro records to the result file
+ */
+public class AvroDeltaInputWriter implements FileDeltaInputWriter<GenericRecord> {
+
+  public static final String AVRO_EXTENSION = ".avro";
+  private static Logger log = Logger.getLogger(AvroDeltaInputWriter.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 WriteStats writeStats;
+  private long recordsWritten = 0;
+
+  // TODO : Handle failure case which may leave behind tons of small corrupt files
+  public AvroDeltaInputWriter(Configuration configuration, String basePath, String schemaStr, Long maxFileSize)
+      throws IOException {
+    this.schema = Schema.parse(schemaStr);
+    this.maxFileSize = maxFileSize;
+    this.configuration = configuration;
+    this.basePath = basePath;
+  }
+
+  @Override
+  public void writeData(GenericRecord iData) throws IOException {
+    this.dataFileWriter.append(iData);
+    recordsWritten++;
+  }
+
+  @Override
+  public void open() throws IOException {
+    Path path = new Path(basePath, new Path(UUID.randomUUID().toString() + AVRO_EXTENSION));
+    this.file = HoodieWrapperFileSystem.convertToHoodiePath(path, configuration);
+    this.fs = (HoodieWrapperFileSystem) this.file
+        .getFileSystem(HoodieParquetWriter.registerFileSystem(path, configuration));
+    this.output = this.fs.create(this.file);
+    this.writer = new GenericDatumWriter(schema);
+    this.dataFileWriter = new DataFileWriter<>(writer).create(schema, output);
+    this.writeStats = new WriteStats();
+  }
+
+  @Override
+  public boolean canWrite() {
+    return fs.getBytesWritten(file) < maxFileSize;
+  }
+
+  @Override
+  public void close() throws IOException {
+    this.writeStats.setBytesWritten(this.fs.getBytesWritten(this.file));
+    this.writeStats.setRecordsWritten(this.recordsWritten);
+    this.writeStats.setFilePath(this.file.toUri().getPath());
+    this.dataFileWriter.close();
+    log.info("New Avro File => " + getPath());
+  }
+
+  @Override
+  public FileDeltaInputWriter getNewWriter() throws IOException {
+    AvroDeltaInputWriter avroDeltaInputWriter = new AvroDeltaInputWriter(this.configuration, this.basePath, this
+        .schema.toString(), this.maxFileSize);
+    avroDeltaInputWriter.open();
+    return avroDeltaInputWriter;
+  }
+
+  public FileSystem getFs() {
+    return fs;
+  }
+
+  public Path getPath() {
+    return this.file;
+  }
+
+  @Override
+  public WriteStats getWriteStats() {
+    return this.writeStats;
+  }
+}
diff --git a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/writer/DeltaInputWriter.java
similarity index 56%
copy from hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
copy to hudi-bench/src/main/java/org/apache/hudi/bench/writer/DeltaInputWriter.java
index c35663e..e16e25d 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/writer/DeltaInputWriter.java
@@ -16,26 +16,34 @@
  * limitations under the License.
  */
 
-package org.apache.hudi;
+package org.apache.hudi.bench.writer;
 
-import java.io.Serializable;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.util.TypedProperties;
+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> {
+
+  /**
+   * Generate any type of data
+   */
+  void writeData(I iData) throws IOException;
 
-  protected transient TypedProperties config;
+  /**
+   * Check whether more data can/should be written to the current instance
+   */
+  boolean canWrite();
 
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
-  }
+  /**
+   * Close the writer so no more data can be written to this instance
+   */
+  void close() throws IOException;
 
   /**
-   * Generate a Hoodie Key out of provided generic record.
+   * Return the write statistics of writing data to this instance
    */
-  public abstract HoodieKey getKey(GenericRecord record);
-}
+  WriteStats getWriteStats();
+}
\ No newline at end of file
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/writer/DeltaWriter.java b/hudi-bench/src/main/java/org/apache/hudi/bench/writer/DeltaWriter.java
new file mode 100644
index 0000000..5c18ff2
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/writer/DeltaWriter.java
@@ -0,0 +1,184 @@
+/*
+ * 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.bench.writer;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.HoodieReadClient;
+import org.apache.hudi.HoodieWriteClient;
+import org.apache.hudi.WriteStatus;
+import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.bench.job.HoodieDeltaStreamerWrapper;
+import org.apache.hudi.bench.job.HoodieTestSuiteJob.HoodieTestSuiteConfig;
+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.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 DeltaWriter {
+
+  private HoodieDeltaStreamerWrapper deltaStreamerWrapper;
+  private HoodieWriteClient writeClient;
+  private HoodieTestSuiteConfig cfg;
+  private Option<String> lastCheckpoint;
+  private HoodieReadClient hoodieReadClient;
+  private transient Configuration configuration;
+  private transient JavaSparkContext sparkContext;
+
+  public DeltaWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteConfig cfg, String schema) throws
+      Exception {
+    this(jsc, props, cfg, schema, true);
+  }
+
+  public DeltaWriter(JavaSparkContext jsc, Properties props, HoodieTestSuiteConfig cfg, String schema,
+      boolean rollbackInflight) throws Exception {
+    // We ensure that only 1 instance of HoodieWriteClient is instantiated for a DeltaWriter
+    // 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;
+  }
+
+  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();
+  }
+
+  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.createNewCommitTime());
+    } else {
+      return Option.of(writeClient.startCommit());
+    }
+  }
+
+  public JavaRDD<WriteStatus> upsert(Option<String> instantTime) throws Exception {
+    if (cfg.useDeltaStreamer) {
+      return deltaStreamerWrapper.upsert(instantTime);
+    } 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(instantTime);
+    } 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(instantTime);
+    } 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(instantTime);
+    } 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 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() throws IllegalAccessException {
+    if (cfg.useDeltaStreamer) {
+      throw new IllegalAccessException("cannot access write client when testing in deltastreamer mode");
+    }
+    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-spark/src/main/java/org/apache/hudi/KeyGenerator.java b/hudi-bench/src/main/java/org/apache/hudi/bench/writer/FileDeltaInputWriter.java
similarity index 58%
copy from hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
copy to hudi-bench/src/main/java/org/apache/hudi/bench/writer/FileDeltaInputWriter.java
index c35663e..4dc6bed 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/writer/FileDeltaInputWriter.java
@@ -16,26 +16,22 @@
  * limitations under the License.
  */
 
-package org.apache.hudi;
+package org.apache.hudi.bench.writer;
 
-import java.io.Serializable;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.util.TypedProperties;
+import java.io.IOException;
 
 /**
- * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record
+ * This extends {@link DeltaInputWriter} and allows to have a file system based output type
  */
-public abstract class KeyGenerator implements Serializable {
+public interface FileDeltaInputWriter<I> extends DeltaInputWriter<I> {
 
-  protected transient TypedProperties config;
-
-  protected KeyGenerator(TypedProperties config) {
-    this.config = config;
-  }
+  /**
+   * Open the path/file to write the data to
+   */
+  void open() throws IOException;
 
   /**
-   * Generate a Hoodie Key out of provided generic record.
+   * Start a new file writer and close the current one
    */
-  public abstract HoodieKey getKey(GenericRecord record);
+  FileDeltaInputWriter<I> getNewWriter() throws IOException;
 }
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/writer/SparkAvroDeltaInputWriter.java b/hudi-bench/src/main/java/org/apache/hudi/bench/writer/SparkAvroDeltaInputWriter.java
new file mode 100644
index 0000000..dcf744a
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/writer/SparkAvroDeltaInputWriter.java
@@ -0,0 +1,65 @@
+/*
+ * 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.bench.writer;
+
+import java.io.IOException;
+import javax.ws.rs.NotSupportedException;
+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 = "com.databricks.spark.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 WriteStats getWriteStats() {
+    throw new NotSupportedException("not applicable for spark based writer");
+  }
+
+}
diff --git a/hudi-bench/src/main/java/org/apache/hudi/bench/writer/WriteStats.java b/hudi-bench/src/main/java/org/apache/hudi/bench/writer/WriteStats.java
new file mode 100644
index 0000000..1ba673e
--- /dev/null
+++ b/hudi-bench/src/main/java/org/apache/hudi/bench/writer/WriteStats.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.bench.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 an instance of {@link DeltaInputWriter}
+ */
+public class WriteStats 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-bench/src/test/java/org/apache/hudi/bench/TestDFSDeltaWriterAdapter.java b/hudi-bench/src/test/java/org/apache/hudi/bench/TestDFSDeltaWriterAdapter.java
new file mode 100644
index 0000000..e9e596a
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/TestDFSDeltaWriterAdapter.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.bench;
+
+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.bench.configuration.DFSDeltaConfig;
+import org.apache.hudi.bench.configuration.DeltaConfig;
+import org.apache.hudi.bench.generator.FlexibleSchemaRecordGenerationIterator;
+import org.apache.hudi.bench.utils.TestUtils;
+import org.apache.hudi.bench.writer.FileDeltaInputWriter;
+import org.apache.hudi.bench.writer.WriteStats;
+import org.apache.hudi.common.SerializableConfiguration;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.utilities.UtilitiesTestBase;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestDFSDeltaWriterAdapter extends UtilitiesTestBase {
+
+  private FilebasedSchemaProvider schemaProvider;
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+  }
+
+  @AfterClass
+  public static void cleanupClass() throws Exception {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS("hudi-bench-config/complex-source.avsc"),
+        jsc);
+  }
+
+  @After
+  public void teardown() throws Exception {
+    super.teardown();
+  }
+
+  @Test
+  public void testDFSOneFileWrite() throws IOException {
+
+    FileDeltaInputWriter<GenericRecord> mockFileSinkWriter = Mockito.mock(FileDeltaInputWriter.class);
+    WriteStats mockWriteStats = Mockito.mock(WriteStats.class);
+    when(mockFileSinkWriter.getNewWriter()).thenReturn(mockFileSinkWriter);
+    when(mockFileSinkWriter.canWrite()).thenReturn(true);
+    when(mockFileSinkWriter.getWriteStats()).thenReturn(mockWriteStats);
+
+    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 {
+
+    FileDeltaInputWriter<GenericRecord> mockFileSinkWriter = Mockito.mock(FileDeltaInputWriter.class);
+    WriteStats mockWriteStats = Mockito.mock(WriteStats.class);
+    when(mockFileSinkWriter.getNewWriter()).thenReturn(mockFileSinkWriter);
+    when(mockFileSinkWriter.canWrite()).thenReturn(false, true);
+    when(mockFileSinkWriter.getWriteStats()).thenReturn(mockWriteStats);
+
+    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(DeltaOutputType.DFS, DeltaInputFormat.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
+    Assert.assertTrue(fileStatuses.length > 0);
+  }
+
+}
diff --git a/hudi-bench/src/test/java/org/apache/hudi/bench/TestFileDeltaInputWriter.java b/hudi-bench/src/test/java/org/apache/hudi/bench/TestFileDeltaInputWriter.java
new file mode 100644
index 0000000..30e0190
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/TestFileDeltaInputWriter.java
@@ -0,0 +1,143 @@
+/*
+ * 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.bench;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.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.bench.generator.GenericRecordFullPayloadGenerator;
+import org.apache.hudi.bench.reader.SparkBasedReader;
+import org.apache.hudi.bench.writer.AvroDeltaInputWriter;
+import org.apache.hudi.bench.writer.FileDeltaInputWriter;
+import org.apache.hudi.bench.writer.WriteStats;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.utilities.UtilitiesTestBase;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestFileDeltaInputWriter extends UtilitiesTestBase {
+
+  private FilebasedSchemaProvider schemaProvider;
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+  }
+
+  @AfterClass
+  public static void cleanupClass() throws Exception {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS("hudi-bench-config/complex-source.avsc"),
+        jsc);
+  }
+
+  @After
+  public void teardown() throws Exception {
+    super.teardown();
+  }
+
+  @Test
+  public void testAvroFileSinkWriter() throws IOException {
+    // 1. Create a Avro File Sink Writer
+    FileDeltaInputWriter<GenericRecord> fileSinkWriter =
+        new AvroDeltaInputWriter(jsc.hadoopConfiguration(), dfsBasePath + "/input", schemaProvider.getSourceSchema()
+            .toString(), 1024 * 1024L);
+    GenericRecordFullPayloadGenerator payloadGenerator =
+        new GenericRecordFullPayloadGenerator(schemaProvider.getSourceSchema());
+    fileSinkWriter.open();
+    // 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();
+    WriteStats writeStats = fileSinkWriter.getWriteStats();
+    FileSystem fs = FSUtils.getFs(dfsBasePath, jsc.hadoopConfiguration());
+    FileStatus[] fileStatuses = fs.listStatus(new Path(writeStats.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(writeStats.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(), writeStats.getRecordsWritten());
+  }
+
+  @Test
+  public void testAvroFileSinkCreateNewWriter() throws IOException {
+    // 1. Create a Avro File Sink Writer
+    FileDeltaInputWriter<GenericRecord> fileSinkWriter =
+        new AvroDeltaInputWriter(jsc.hadoopConfiguration(), dfsBasePath, schemaProvider.getSourceSchema().toString(),
+            1024 * 1024L);
+    GenericRecordFullPayloadGenerator payloadGenerator =
+        new GenericRecordFullPayloadGenerator(schemaProvider.getSourceSchema());
+    fileSinkWriter.open();
+    // 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.getWriteStats().getFilePath();
+    assertFalse(oldFilePath == null);
+    FileDeltaInputWriter<GenericRecord> newFileSinkWriter = fileSinkWriter.getNewWriter();
+    newFileSinkWriter.close();
+    WriteStats newStats = newFileSinkWriter.getWriteStats();
+    assertEquals(newStats.getBytesWritten(), 3674);
+    assertEquals(newStats.getRecordsWritten(), 0);
+    assertTrue(newStats.getFilePath() != null);
+  }
+
+}
diff --git a/hudi-bench/src/test/java/org/apache/hudi/bench/configuration/TestWorkflowBuilder.java b/hudi-bench/src/test/java/org/apache/hudi/bench/configuration/TestWorkflowBuilder.java
new file mode 100644
index 0000000..77cc42c
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/configuration/TestWorkflowBuilder.java
@@ -0,0 +1,76 @@
+/*
+ * 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.bench.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.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.WorkflowDag;
+import org.apache.hudi.bench.dag.nodes.DagNode;
+import org.apache.hudi.bench.dag.nodes.InsertNode;
+import org.apache.hudi.bench.dag.nodes.UpsertNode;
+import org.junit.Test;
+
+public class TestWorkflowBuilder {
+
+  @Test
+  public void testWorkloadOperationSequenceBuilder() {
+
+    DagNode root = new InsertNode(Config.newBuilder()
+        .withNumRecordsToInsert(10000)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(2)
+        .withRecordSize(1000).build());
+
+    DagNode child1 = new UpsertNode(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);
+    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-bench/src/test/java/org/apache/hudi/bench/converter/TestUpdateConverter.java b/hudi-bench/src/test/java/org/apache/hudi/bench/converter/TestUpdateConverter.java
new file mode 100644
index 0000000..cb476d7
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/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.bench.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.bench.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.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import scala.Tuple2;
+
+public class TestUpdateConverter {
+
+  private JavaSparkContext jsc;
+
+  @Before
+  public void setup() throws Exception {
+    jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[1]");
+
+  }
+
+  @After
+  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();
+    Assert.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
+    Assert.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-bench/src/test/java/org/apache/hudi/bench/dag/TestComplexDag.java b/hudi-bench/src/test/java/org/apache/hudi/bench/dag/TestComplexDag.java
new file mode 100644
index 0000000..c9492b4
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/dag/TestComplexDag.java
@@ -0,0 +1,78 @@
+/*
+ * 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.bench.dag;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+import org.apache.hudi.WriteStatus;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.nodes.DagNode;
+import org.apache.hudi.bench.dag.nodes.InsertNode;
+import org.apache.hudi.bench.dag.nodes.UpsertNode;
+import org.apache.hudi.bench.dag.nodes.ValidateNode;
+import org.apache.spark.api.java.JavaRDD;
+
+public class TestComplexDag extends 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(3)
+        .withNumUpsertPartitions(3)
+        .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();
+      assert b1 && b2 && b3 == true;
+      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-bench/src/test/java/org/apache/hudi/bench/dag/TestDagUtils.java b/hudi-bench/src/test/java/org/apache/hudi/bench/dag/TestDagUtils.java
new file mode 100644
index 0000000..afeaf6a
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/dag/TestDagUtils.java
@@ -0,0 +1,90 @@
+/*
+ * 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.bench.dag;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.nodes.DagNode;
+import org.apache.hudi.bench.dag.nodes.InsertNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestDagUtils {
+
+  @Test
+  public void testConvertDagToYaml() throws Exception {
+    ComplexDag dag = new ComplexDag();
+    String yaml = DagUtils.convertDagToYaml(dag.build());
+    System.out.println(yaml);
+  }
+
+  @Test
+  public void testConvertYamlToDag() throws Exception {
+    WorkflowDag dag = DagUtils.convertYamlToDag(
+        DagUtils.toString(getClass().getResourceAsStream("/hudi-bench-config/complex-workflow-dag-cow.yaml")));
+    Assert.assertEquals(dag.getNodeList().size(), 1);
+    Assert.assertEquals(((DagNode) dag.getNodeList().get(0)).getParentNodes().size(), 0);
+    Assert.assertEquals(((DagNode) dag.getNodeList().get(0)).getChildNodes().size(), 1);
+    DagNode firstChild = (DagNode) ((DagNode) dag.getNodeList().get(0)).getChildNodes().get(0);
+    Assert.assertEquals(firstChild.getParentNodes().size(), 1);
+    Assert.assertEquals(firstChild.getChildNodes().size(), 1);
+    Assert.assertEquals(((DagNode) firstChild.getChildNodes().get(0)).getChildNodes().size(), 1);
+  }
+
+  public static class ComplexDag extends 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-bench/src/test/java/org/apache/hudi/bench/dag/TestHiveSyncDag.java b/hudi-bench/src/test/java/org/apache/hudi/bench/dag/TestHiveSyncDag.java
new file mode 100644
index 0000000..17c9730
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/dag/TestHiveSyncDag.java
@@ -0,0 +1,55 @@
+/*
+ * 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.bench.dag;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.nodes.DagNode;
+import org.apache.hudi.bench.dag.nodes.HiveQueryNode;
+import org.apache.hudi.bench.dag.nodes.HiveSyncNode;
+import org.apache.hudi.bench.dag.nodes.InsertNode;
+import org.apache.hudi.common.util.collection.Pair;
+
+public class TestHiveSyncDag extends 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.hive_trips 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-bench/src/test/java/org/apache/hudi/bench/dag/TestInsertOnlyDag.java b/hudi-bench/src/test/java/org/apache/hudi/bench/dag/TestInsertOnlyDag.java
new file mode 100644
index 0000000..f50fe97
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/dag/TestInsertOnlyDag.java
@@ -0,0 +1,50 @@
+/*
+ * 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.bench.dag;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.nodes.DagNode;
+import org.apache.hudi.bench.dag.nodes.InsertNode;
+
+public class TestInsertOnlyDag extends WorkflowDagGenerator {
+
+  @Override
+  public WorkflowDag build() {
+    DagNode root = new InsertNode(Config.newBuilder()
+        .withNumRecordsToInsert(100)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(2)
+        .withRecordSize(1000).build());
+
+    DagNode child1 = new InsertNode(Config.newBuilder()
+        .withNumRecordsToInsert(100)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(2)
+        .withRecordSize(1000).build());
+
+    root.addChildNode(child1);
+    child1.addParentNode(root);
+    List<DagNode> rootNodes = new ArrayList<>();
+    rootNodes.add(root);
+    return new WorkflowDag(rootNodes);
+  }
+
+}
diff --git a/hudi-bench/src/test/java/org/apache/hudi/bench/dag/TestInsertUpsertDag.java b/hudi-bench/src/test/java/org/apache/hudi/bench/dag/TestInsertUpsertDag.java
new file mode 100644
index 0000000..63d898d
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/dag/TestInsertUpsertDag.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bench.dag;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+import org.apache.hudi.bench.dag.nodes.DagNode;
+import org.apache.hudi.bench.dag.nodes.InsertNode;
+import org.apache.hudi.bench.dag.nodes.UpsertNode;
+
+public class TestInsertUpsertDag extends WorkflowDagGenerator {
+
+  @Override
+  public WorkflowDag build() {
+    DagNode root = new InsertNode(Config.newBuilder()
+        .withNumRecordsToInsert(100)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(2)
+        .withRecordSize(1000).build());
+
+    DagNode child1 = new UpsertNode(Config.newBuilder()
+        .withNumRecordsToUpdate(100)
+        .withNumInsertPartitions(1)
+        .withNumTimesToRepeat(2)
+        .withRecordSize(1000).build());
+
+    root.addChildNode(child1);
+    child1.addParentNode(root);
+    List<DagNode> rootNodes = new ArrayList<>();
+    rootNodes.add(root);
+    return new WorkflowDag(rootNodes);
+  }
+
+}
diff --git a/hudi-bench/src/test/java/org/apache/hudi/bench/generator/TestGenericRecordPayloadEstimator.java b/hudi-bench/src/test/java/org/apache/hudi/bench/generator/TestGenericRecordPayloadEstimator.java
new file mode 100644
index 0000000..e616813
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/generator/TestGenericRecordPayloadEstimator.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bench.generator;
+
+import static junit.framework.TestCase.assertEquals;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.common.util.collection.Pair;
+import org.junit.Test;
+
+public class TestGenericRecordPayloadEstimator {
+
+  @Test
+  public void testSimpleSchemaSize() throws Exception {
+    Schema schema = new Schema.Parser().parse(getClass().getClassLoader()
+        .getResourceAsStream("hudi-bench-config/source.avsc"));
+    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(getClass().getClassLoader()
+        .getResourceAsStream("hudi-bench-config/complex-source.avsc"));
+    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-bench/src/test/java/org/apache/hudi/bench/generator/TestGenericRecordPayloadGenerator.java b/hudi-bench/src/test/java/org/apache/hudi/bench/generator/TestGenericRecordPayloadGenerator.java
new file mode 100644
index 0000000..e2c3ae6
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/generator/TestGenericRecordPayloadGenerator.java
@@ -0,0 +1,122 @@
+/*
+ * 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.bench.generator;
+
+import static junit.framework.TestCase.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.common.util.HoodieAvroUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestGenericRecordPayloadGenerator {
+
+  @Test
+  public void testSimplePayload() throws Exception {
+    Schema schema = new Schema.Parser().parse(getClass().getClassLoader()
+        .getResourceAsStream("hudi-bench-config/source.avsc"));
+    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(getClass().getClassLoader()
+        .getResourceAsStream("hudi-bench-config/complex-source.avsc"));
+    GenericRecordFullPayloadGenerator payloadGenerator = new GenericRecordFullPayloadGenerator(schema);
+    GenericRecord record = payloadGenerator.getNewPayload();
+    // The generated payload should validate with the provided schema
+    Assert.assertTrue(payloadGenerator.validate(record));
+  }
+
+  @Test
+  public void testComplexPartialPayload() throws IOException {
+    Schema schema = new Schema.Parser().parse(getClass().getClassLoader()
+        .getResourceAsStream("hudi-bench-config/complex-source.avsc"));
+    GenericRecordPartialPayloadGenerator payloadGenerator = new GenericRecordPartialPayloadGenerator(schema);
+    IntStream.range(0, 10).forEach(a -> {
+      GenericRecord record = payloadGenerator.getNewPayload();
+      // The generated payload should validate with the provided schema
+      Assert.assertTrue(payloadGenerator.validate(record));
+    });
+  }
+
+  @Test
+  public void testUpdatePayloadGenerator() throws IOException {
+    Schema schema = new Schema.Parser().parse(getClass().getClassLoader()
+        .getResourceAsStream("hudi-bench-config/source.avsc"));
+    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
+    Assert.assertTrue(insertRowKeys.containsAll(updateRowKeys));
+    // The timestamp field for the insert payloads should not all match with the update payloads
+    Assert.assertFalse(insertTimeStamps.containsAll(updateTimeStamps));
+  }
+
+  @Test
+  public void testSimplePayloadWithLargeMinSize() throws Exception {
+    Schema schema = new Schema.Parser().parse(getClass().getClassLoader()
+        .getResourceAsStream("hudi-bench-config/source.avsc"));
+    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(getClass().getClassLoader()
+        .getResourceAsStream("hudi-bench-config/complex-source.avsc"));
+    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-bench/src/test/java/org/apache/hudi/bench/generator/TestWorkloadGenerator.java b/hudi-bench/src/test/java/org/apache/hudi/bench/generator/TestWorkloadGenerator.java
new file mode 100644
index 0000000..5f10005
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/generator/TestWorkloadGenerator.java
@@ -0,0 +1,372 @@
+/*
+ * 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.bench.generator;///*
+// *  Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+// *
+// *  Licensed 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.bench.generator;
+//
+//import static junit.framework.TestCase.assertEquals;
+//import static junit.framework.TestCase.assertFalse;
+//import static org.mockito.Mockito.when;
+//
+//import org.apache.hudi.bench.configuration.DeltaConfig.Config;
+//import org.apache.hudi.bench.input.config.DFSDeltaConfig;
+//import org.apache.hudi.bench.input.config.DeltaConfig;
+//import org.apache.hudi.common.SerializableConfiguration;
+//import org.apache.hudi.common.util.FSUtils;
+//import org.apache.hudi.bench.job.operation.Operation;
+//import org.apache.hudi.bench.input.DeltaInputFormat;
+//import org.apache.hudi.bench.input.DeltaOutputType;
+//import org.apache.hudi.bench.input.reader.DFSAvroDeltaInputReader;
+//import org.apache.hudi.bench.utils.TestUtils;
+//import org.apache.hudi.utilities.UtilitiesTestBase;
+//import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+//import java.io.IOException;
+//import java.util.Arrays;
+//import java.util.Collections;
+//import java.util.HashMap;
+//import java.util.Map;
+//import org.apache.hudi.common.util.Option;
+//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.spark.api.java.JavaRDD;
+//import org.junit.After;
+//import org.junit.AfterClass;
+//import org.junit.Before;
+//import org.junit.BeforeClass;
+//import org.junit.Ignore;
+//import org.junit.Test;
+//import org.mockito.Mockito;
+//import org.powermock.api.mockito.PowerMockito;
+//import org.powermock.core.classloader.annotations.PrepareForTest;
+//
+//// NOTE : Need the following to ensure that local objects are mocked using PowerMockito but this clashes with
+//// HDFSMiniCluster setup by reloading some classes leading to incorrect path/permissions issue
+//// @RunWith(PowerMockRunner.class)
+//// @PowerMockIgnore({"org.apache.apache._", "com.sun.*"})
+//@PrepareForTest({DFSAvroDeltaInputReader.class})
+//public class TestWorkloadGenerator extends UtilitiesTestBase {
+//
+//  private FilebasedSchemaProvider schemaProvider;
+//
+//  @BeforeClass
+//  public static void initClass() throws Exception {
+//    UtilitiesTestBase.initClass();
+//  }
+//
+//  @AfterClass
+//  public static void cleanupClass() throws Exception {
+//    UtilitiesTestBase.cleanupClass();
+//  }
+//
+//  @Before
+//  public void setup() throws Exception {
+//    super.setup();
+//    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS("hudi-bench-config/source.avsc"),
+//        jsc);
+//  }
+//
+//  @After
+//  public void teardown() throws Exception {
+//    super.teardown();
+//  }
+//
+//  @Test
+//  @Ignore
+//  public void testInsertWorkloadGenerator() throws Exception {
+//    LazyRecordGeneratorIterator mockLazyRecordGeneratorIterator = Mockito.mock(LazyRecordGeneratorIterator.class);
+//    when(mockLazyRecordGeneratorIterator.hasNext()).thenReturn(true, true, true, true, true, false);
+//    PowerMockito.whenNew(LazyRecordGeneratorIterator.class)
+//        .withArguments(new FlexibleSchemaRecordGenerationIterator(5, schemaProvider.getSourceSchema()
+//            .toString())).thenReturn(mockLazyRecordGeneratorIterator);
+//    // Perform upserts of all 5 records
+//    Config dataGenerationConfig =
+//        new Config(Operation.INSERT, 5, 0, 1, 1024, 1, Option.empty(), false);
+//    DeltaConfig mockSinkConfig = Mockito.mock(DeltaConfig.class);
+//    DeltaGenerator workloadGenerator = new DeltaGenerator(mockSinkConfig, );
+//    JavaRDD<GenericRecord> inputRDD = workloadGenerator.generateInserts(jsc, dataGenerationConfig, schemaProvider
+//        .getSourceSchema().toString(), Collections.emptyList());
+//    assertEquals(inputRDD.count(), 5);
+//  }
+//
+//  @Test
+//  public void testGetPartitionToCountMap() {
+//    DFSDeltaConfig mockSinkConfig = Mockito.mock(DFSDeltaConfig.class);
+//    DeltaGenerator workloadGenerator = new DeltaGenerator(mockSinkConfig, );
+//    Long actualNumRecords = 5L;
+//    JavaRDD<GenericRecord> rdd = TestUtils.makeRDD(jsc, actualNumRecords.intValue());
+//    // Test for 1 spark partition
+//    int numPartitions = rdd.getNumPartitions();
+//    Map<Integer, Long> map = workloadGenerator.getPartitionToCountMap(rdd);
+//    // Total num spark partitions should be 1
+//    assertEquals(map.size(), numPartitions);
+//    // Total records in 1 num spark partition should be 5
+//    Long totalRecord = map.get(0);
+//    assertEquals(totalRecord, actualNumRecords);
+//    // Test for 2 spark partitions
+//    rdd = rdd.repartition(2);
+//    numPartitions = rdd.getNumPartitions();
+//    map = workloadGenerator.getPartitionToCountMap(rdd);
+//    // Total num spark partitions should be 2
+//    assertEquals(map.size(), numPartitions);
+//    // Total records in 2 num spark partitions should be 5
+//    totalRecord = map.get(0) + map.get(1);
+//    assertEquals(totalRecord, actualNumRecords);
+//  }
+//
+//  @Test
+//  public void testGetAdjustedPartitionsCount() {
+//    DFSDeltaConfig mockSinkConfig = Mockito.mock(DFSDeltaConfig.class);
+//    DeltaGenerator workloadGenerator = new DeltaGenerator(mockSinkConfig, );
+//    // Test for 1 spark partition
+//    Map<Integer, Long> partitionCountMap = new HashMap<>();
+//    long totalInitRecords = 10L;
+//    partitionCountMap.put(0, totalInitRecords);
+//    Map<Integer, Long> adjustedPartitionsCount = workloadGenerator.getAdjustedPartitionsCount(partitionCountMap, 5);
+//    assertEquals(adjustedPartitionsCount.size(), 1);
+//    Long newCount = adjustedPartitionsCount.get(0);
+//    Long totalNewRecordsExpected = 5L;
+//    assertEquals(newCount, totalNewRecordsExpected);
+//    // Test for 3 spark partitions
+//    partitionCountMap = new HashMap<>();
+//    partitionCountMap.put(0, 3L);
+//    partitionCountMap.put(1, 4L);
+//    partitionCountMap.put(2, 3L);
+//    adjustedPartitionsCount = workloadGenerator.getAdjustedPartitionsCount(partitionCountMap, 5);
+//    // Should remove 1 partition completely since all partitions have count less than five
+//    assertEquals(adjustedPartitionsCount.size(), 2);
+//    // Should decrement 1 partition to remove total of 5 records
+//    Map.Entry<Integer, Long> partitionToAdjust = adjustedPartitionsCount.entrySet().iterator().next();
+//    newCount = partitionToAdjust.getValue();
+//    Long originalValueForPartition = partitionCountMap.get(partitionToAdjust.getKey());
+//    assertFalse(newCount == originalValueForPartition);
+//  }
+//
+//  @Test
+//  public void testAdjustRDDToGenerateExactNumUpdates() {
+//    DFSDeltaConfig mockSinkConfig = Mockito.mock(DFSDeltaConfig.class);
+//    DeltaGenerator workloadGenerator = new DeltaGenerator(mockSinkConfig, );
+//    long totalRecordsRequired = 10;
+//    JavaRDD<GenericRecord> updates = TestUtils.makeRDD(jsc, 5);
+//    // Test flow to generate more updates
+//    JavaRDD<GenericRecord> adjustedRDD = workloadGenerator.adjustRDDToGenerateExactNumUpdates(updates, jsc,
+//        totalRecordsRequired);
+//    assertEquals(adjustedRDD.count(), totalRecordsRequired);
+//    totalRecordsRequired = 100;
+//    adjustedRDD = workloadGenerator.adjustRDDToGenerateExactNumUpdates(updates, jsc, totalRecordsRequired);
+//    assertEquals(adjustedRDD.count(), totalRecordsRequired);
+//    // Test flow to generate less updates
+//    totalRecordsRequired = 3;
+//    adjustedRDD = workloadGenerator.adjustRDDToGenerateExactNumUpdates(updates, jsc,
+//        totalRecordsRequired);
+//    assertEquals(adjustedRDD.count(), totalRecordsRequired);
+//  }
+//
+//  @Test
+//  public void testDFSAvroWorkloadGeneratorSimple() throws IOException {
+//    // Perform inserts of 1000 records
+//    DeltaConfig sinkConfig = new DFSDeltaConfig(DeltaOutputType.DFS, DeltaInputFormat.AVRO,
+//        new SerializableConfiguration(jsc.hadoopConfiguration()), dfsBasePath, schemaProvider.getSourceSchema()
+//        .toString(), 1024 * 1024L);
+//    DeltaGenerator workloadGenerator = new DeltaGenerator(sinkConfig, );
+//    Config dataGenerationConfig =
+//        new Config(Operation.INSERT, 1000, 0, 1, 1024, 1, Option.empty(), false);
+//    JavaRDD<GenericRecord> inputRDD = workloadGenerator
+//        .generateInserts(jsc, dataGenerationConfig, schemaProvider.getSourceSchema().toString(), Collections
+//            .emptyList());
+//    assertEquals(inputRDD.count(), 1000);
+//    // Write 1 file for all these inserts
+//    assertEquals(workloadGenerator.writeRecords(inputRDD, 1).collect().size(), 1);
+//    // Perform upserts of all 1000 records
+//    dataGenerationConfig =
+//        new Config(Operation.UPSERT, 0, 1000, 1, 1024, 1, Option.empty(), false);
+//    inputRDD = workloadGenerator
+//        .generateUpdates(jsc, sparkSession, dataGenerationConfig,
+//            Arrays.asList("_row_key"), Collections.emptyList(), schemaProvider.getSourceSchema().toString());
+//    assertEquals(inputRDD.count(), 1000);
+//    // Write 1 file for all these upserts
+//    assertEquals(workloadGenerator.writeRecords(inputRDD, 1).collect().size(), 1);
+//    FileSystem fs = FSUtils.getFs(dfsBasePath, sinkConfig.getConfiguration());
+//    FileStatus[] fileStatuses = fs.globStatus(new Path(dfsBasePath + "/*/*.avro"));
+//    // 2 files should be present, 1 for inserts and 1 for upserts
+//    assertEquals(fileStatuses.length, 2);
+//  }
+//
+//  @Test
+//  public void testGenerateUpdatesFromGreaterNumExistingInserts() throws IOException {
+//    // Perform inserts of 1000 records
+//    DeltaConfig sinkConfig = new DFSDeltaConfig(DeltaOutputType.DFS, DeltaInputFormat.AVRO,
+//        new SerializableConfiguration(jsc.hadoopConfiguration()), dfsBasePath, schemaProvider.getSourceSchema()
+//        .toString(), 1024 * 1024L);
+//    DeltaGenerator workloadGenerator = new DeltaGenerator(sinkConfig, );
+//    Config dataGenerationConfig =
+//        new Config(Operation.INSERT, 1000, 0, 1, 1024, 1, Option.empty(), false);
+//    JavaRDD<GenericRecord> inputRDD = workloadGenerator
+//        .generateInserts(jsc, dataGenerationConfig, schemaProvider.getSourceSchema().toString(), Collections
+//            .emptyList());
+//    assertEquals(workloadGenerator.writeRecords(inputRDD, 1).collect().size(), 1);
+//    // Perform upserts of 500 records
+//    dataGenerationConfig =
+//        new Config(Operation.UPSERT, 0, 1000, 1, 1024, 1, Option.empty(), false);
+//    inputRDD = workloadGenerator
+//        .generateUpdates(jsc, sparkSession, dataGenerationConfig,
+//            Arrays.asList("_row_key"), Collections.emptyList(), schemaProvider.getSourceSchema().toString());
+//    // We should be able to generate 500 updates from the previously inserted 1000 records
+//    assertEquals(inputRDD.count(), 1000);
+//  }
+//
+//  @Test
+//  public void testGenerateUpdatesFromSmallerNumExistingInserts() throws IOException {
+//    // Perform inserts of 1000 records
+//    DeltaConfig sinkConfig = new DFSDeltaConfig(DeltaOutputType.DFS, DeltaInputFormat.AVRO,
+//        new SerializableConfiguration(jsc.hadoopConfiguration()), dfsBasePath, schemaProvider.getSourceSchema()
+//        .toString(), 1024 * 1024L);
+//    DeltaGenerator workloadGenerator = new DeltaGenerator(sinkConfig, );
+//    Config dataGenerationConfig =
+//        new Config(Operation.INSERT, 500, 0, 1, 1024, 1, Option.empty(), false);
+//    JavaRDD<GenericRecord> inputRDD = workloadGenerator
+//        .generateInserts(jsc, dataGenerationConfig, schemaProvider.getSourceSchema().toString(), Collections
+//            .emptyList());
+//    // Ensure that 500 inserts were written to 1 file
+//    assertEquals(workloadGenerator.writeRecords(inputRDD, 1).collect().size(), 1);
+//    // Perform upserts of 1000 records
+//    dataGenerationConfig =
+//        new Config(Operation.UPSERT, 0, 1000, 1, 1024, 1, Option.empty(), false);
+//    inputRDD = workloadGenerator
+//        .generateUpdates(jsc, sparkSession, dataGenerationConfig,
+//            Arrays.asList("_row_key"), Collections.emptyList(), schemaProvider.getSourceSchema().toString());
+//    // We should be able to generate 1000 updates from the previously inserted 500 records
+//    assertEquals(inputRDD.count(), 1000);
+//  }
+//
+//  @Test
+//  public void testGenerateUpdatesFromMuchSmallerNumExistingInserts() throws IOException {
+//    // Perform inserts of 1000 records
+//    DeltaConfig sinkConfig = new DFSDeltaConfig(DeltaOutputType.DFS, DeltaInputFormat.AVRO,
+//        new SerializableConfiguration(jsc.hadoopConfiguration()), dfsBasePath, schemaProvider.getSourceSchema()
+//        .toString(), 1024 * 1024L);
+//    DeltaGenerator workloadGenerator = new DeltaGenerator(sinkConfig, );
+//    Config dataGenerationConfig =
+//        new Config(Operation.INSERT, 1000, 0, 1, 1024, 1, Option.empty(), false);
+//    JavaRDD<GenericRecord> inputRDD = workloadGenerator
+//        .generateInserts(jsc, dataGenerationConfig, schemaProvider.getSourceSchema().toString(), Collections
+//            .emptyList());
+//    // Ensure that 1000 inserts were written to 1 file
+//    assertEquals(workloadGenerator.writeRecords(inputRDD, 1).collect().size(), 1);
+//    // Perform upserts of 10000 records
+//    dataGenerationConfig =
+//        new Config(Operation.UPSERT, 0, 10000, 1, 1024, 1, Option.empty(), false);
+//    inputRDD = workloadGenerator
+//        .generateUpdates(jsc, sparkSession, dataGenerationConfig,
+//            Arrays.asList("_row_key"), Collections.emptyList(), schemaProvider.getSourceSchema().toString());
+//    // We should be able to generate 10000 updates from the previously inserted 1000 records
+//    assertEquals(inputRDD.count(), 10000);
+//  }
+//
+//  @Test
+//  public void testAdjustUpdatesFromGreaterNumUpdates() throws IOException {
+//    // Perform inserts of 1000 records into 3 files
+//    DeltaConfig sinkConfig = new DFSDeltaConfig(DeltaOutputType.DFS, DeltaInputFormat.AVRO,
+//        new SerializableConfiguration(jsc.hadoopConfiguration()), dfsBasePath, schemaProvider.getSourceSchema()
+//        .toString(), 10240L);
+//    DeltaGenerator workloadGenerator = new DeltaGenerator(sinkConfig, );
+//    Config dataGenerationConfig =
+//        new Config(Operation.INSERT, 1000, 0, 1, 1024, 1, Option.empty(), false);
+//    JavaRDD<GenericRecord> inputRDD = workloadGenerator
+//        .generateInserts(jsc, dataGenerationConfig, schemaProvider.getSourceSchema().toString(), Collections
+//            .emptyList());
+//    assertEquals(workloadGenerator.writeRecords(inputRDD, 1).collect().size(), 3);
+//    // Perform upsert of 450 records. This will force us to readAvro min (3) files with more than 450 records readAvro
+//    dataGenerationConfig =
+//        new Config(Operation.UPSERT, 0, 450, 1, 1024, 1, Option.empty(), false);
+//    inputRDD = workloadGenerator
+//        .generateUpdates(jsc, sparkSession, dataGenerationConfig,
+//            Arrays.asList("_row_key"), Collections.emptyList(), schemaProvider.getSourceSchema().toString());
+//    // We should be able to generate 450 updates from the previously inserted 1000 records
+//    assertEquals(inputRDD.count(), 450);
+//  }
+//
+//  @Test
+//  public void testUpdateGeneratorForNoPartitions() throws IOException {
+//    // Perform inserts of 1000 records into 3 files
+//    DeltaConfig sinkConfig = new DFSDeltaConfig(DeltaOutputType.DFS, DeltaInputFormat.AVRO,
+//        new SerializableConfiguration(jsc.hadoopConfiguration()), dfsBasePath, schemaProvider.getSourceSchema()
+//        .toString(), 10240L);
+//    DeltaGenerator workloadGenerator = new DeltaGenerator(sinkConfig, );
+//    Config dataGenerationConfig =
+//        new Config(Operation.INSERT, 1000, 0, 1, 1024, 1, Option.empty(), false);
+//    JavaRDD<GenericRecord> inputRDD = workloadGenerator
+//        .generateInserts(jsc, dataGenerationConfig, schemaProvider.getSourceSchema().toString(), Collections
+//            .emptyList());
+//    assertEquals(workloadGenerator.writeRecords(inputRDD, 1).collect().size(), 3);
+//    // Perform upsert of 450 records. This will force us to readAvro min (3) files with more than 450 records readAvro
+//    dataGenerationConfig =
+//        new Config(Operation.UPSERT, 0, 450, 1, 1024, 0, Option.empty(), false);
+//    inputRDD = workloadGenerator
+//        .generateUpdates(jsc, sparkSession, dataGenerationConfig,
+//            Arrays.asList("_row_key"), Collections.emptyList(), schemaProvider.getSourceSchema().toString());
+//    assertEquals(inputRDD.getNumPartitions(), 1);
+//    // We should be able to generate 150 updates from the previously inserted 1000 records
+//    assertEquals(inputRDD.count(), 450);
+//
+//  }
+//
+//  @Test
+//  public void testUpdateGeneratorForInvalidPartitionFieldNames() throws IOException {
+//    // Perform inserts of 1000 records into 3 files
+//    DeltaConfig sinkConfig = new DFSDeltaConfig(DeltaOutputType.DFS, DeltaInputFormat.AVRO,
+//        new SerializableConfiguration(jsc.hadoopConfiguration()), dfsBasePath, schemaProvider.getSourceSchema()
+//        .toString(), 10240L);
+//    DeltaGenerator workloadGenerator = new DeltaGenerator(sinkConfig, );
+//    Config dataGenerationConfig =
+//        new Config(Operation.INSERT, 1000, 0, 1, 1024, 1, Option.empty(), false);
+//    JavaRDD<GenericRecord> inputRDD = workloadGenerator
+//        .generateInserts(jsc, dataGenerationConfig, schemaProvider.getSourceSchema().toString(), Collections
+//            .emptyList());
+//    assertEquals(workloadGenerator.writeRecords(inputRDD, 1).collect().size(), 3);
+//    // Perform upsert of 450 records. This will force us to readAvro min (3) files with more than 450 records readAvro
+//    dataGenerationConfig =
+//        new Config(Operation.UPSERT, 0, 450, 1, 1024, 0, Option.empty(), false);
+//    inputRDD = workloadGenerator
+//        .generateUpdates(jsc, sparkSession, dataGenerationConfig,
+//            Arrays.asList("_row_key"), Arrays.asList("not_there"), schemaProvider.getSourceSchema().toString());
+//    assertEquals(inputRDD.getNumPartitions(), 1);
+//    // We should be able to generate 450 updates from the previously inserted 1000 records
+//    assertEquals(inputRDD.count(), 450);
+//
+//  }
+//
+//}
diff --git a/hudi-bench/src/test/java/org/apache/hudi/bench/job/TestHoodieTestSuiteJob.java b/hudi-bench/src/test/java/org/apache/hudi/bench/job/TestHoodieTestSuiteJob.java
new file mode 100644
index 0000000..c2ef8ba
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/job/TestHoodieTestSuiteJob.java
@@ -0,0 +1,203 @@
+/*
+ * 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.bench.job;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.bench.DeltaInputFormat;
+import org.apache.hudi.bench.DeltaOutputType;
+import org.apache.hudi.bench.dag.TestHiveSyncDag;
+import org.apache.hudi.bench.dag.TestInsertOnlyDag;
+import org.apache.hudi.bench.dag.TestInsertUpsertDag;
+import org.apache.hudi.bench.dag.WorkflowDagGenerator;
+import org.apache.hudi.bench.job.HoodieTestSuiteJob.HoodieTestSuiteConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.TypedProperties;
+import org.apache.hudi.utilities.UtilitiesTestBase;
+import org.apache.hudi.utilities.keygen.TimestampBasedKeyGenerator;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.hudi.utilities.sources.AvroDFSSource;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestHoodieTestSuiteJob extends UtilitiesTestBase {
+
+  String tableType;
+  boolean useDeltaStream;
+
+  public TestHoodieTestSuiteJob(String tableType, boolean useDeltaStream) {
+    this.tableType = tableType;
+    this.useDeltaStream = useDeltaStream;
+  }
+
+  @Parameterized.Parameters(name = "TableType")
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[][]{{"COPY_ON_WRITE", false}});
+  }
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+    // prepare the configs.
+    UtilitiesTestBase.Helpers.copyToDFS(UtilitiesTestBase.Helpers.class.getClassLoader(), "hudi-bench-config/base"
+            + ".properties", dfs, dfsBasePath + "/base.properties");
+    UtilitiesTestBase.Helpers.copyToDFS(UtilitiesTestBase.Helpers.class.getClassLoader(), "hudi-bench-config/source"
+        + ".avsc", dfs, dfsBasePath + "/source.avsc");
+    UtilitiesTestBase.Helpers.copyToDFS(UtilitiesTestBase.Helpers.class.getClassLoader(), "hudi-bench-config/target"
+        + ".avsc", dfs, dfsBasePath + "/target.avsc");
+
+    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");
+    // 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(), "hive_trips");
+    props.setProperty(DataSourceWriteOptions.HIVE_ASSUME_DATE_PARTITION_OPT_KEY(), "false");
+    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");
+  }
+
+  @AfterClass
+  public static void cleanupClass() throws Exception {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+  }
+
+  @After
+  public void teardown() throws Exception {
+    super.teardown();
+  }
+
+  // TODO : Clean up input / result paths after each test
+  @Test
+  public void testSimpleInsert() 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);
+    cfg.workloadDagGenerator = TestInsertOnlyDag.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);
+  }
+
+  @Test
+  public void testSimpleInsertUpdate() 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);
+    cfg.workloadDagGenerator = TestInsertUpsertDag.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);
+  }
+
+  @Test
+  public void testComplexDag() 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);
+    cfg.workloadDagGenerator = WorkflowDagGenerator.class.getName();
+    HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc);
+    hoodieTestSuiteJob.runTestSuite();
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(new Configuration(), cfg.targetBasePath);
+    assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 3);
+  }
+
+  @Test
+  public void testHiveSync() 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);
+    cfg.workloadDagGenerator = TestHiveSyncDag.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);
+  }
+
+  protected HoodieTestSuiteConfig makeConfig(String inputBasePath, String outputBasePath) {
+    HoodieTestSuiteConfig cfg = new HoodieTestSuiteConfig();
+    cfg.targetBasePath = outputBasePath;
+    cfg.inputBasePath = inputBasePath;
+    cfg.targetTableName = "hoodie_trips";
+    cfg.storageType = this.tableType;
+    cfg.sourceClassName = AvroDFSSource.class.getName();
+    cfg.sourceOrderingField = "timestamp";
+    cfg.propsFilePath = dfsBasePath + "/test-source.properties";
+    cfg.outputTypeName = DeltaOutputType.DFS.name();
+    cfg.inputFormatName = DeltaInputFormat.AVRO.name();
+    cfg.limitFileSize = 1024 * 1024L;
+    cfg.sourceLimit = 20000000;
+    cfg.workloadDagGenerator = WorkflowDagGenerator.class.getName();
+    cfg.schemaProviderClassName = FilebasedSchemaProvider.class.getName();
+    cfg.useDeltaStreamer = this.useDeltaStream;
+    return cfg;
+  }
+
+}
diff --git a/hudi-bench/src/test/java/org/apache/hudi/bench/reader/TestDFSAvroDeltaInputReader.java b/hudi-bench/src/test/java/org/apache/hudi/bench/reader/TestDFSAvroDeltaInputReader.java
new file mode 100644
index 0000000..9cfb70f
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/reader/TestDFSAvroDeltaInputReader.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bench.reader;
+
+import static junit.framework.Assert.assertTrue;
+import static junit.framework.TestCase.assertEquals;
+
+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.bench.utils.TestUtils;
+import org.apache.hudi.common.util.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.utilities.UtilitiesTestBase;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestDFSAvroDeltaInputReader extends UtilitiesTestBase {
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+  }
+
+  @AfterClass
+  public static void cleanupClass() throws Exception {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+  }
+
+  @Test
+  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-bench/src/test/java/org/apache/hudi/bench/reader/TestDFSHoodieDatasetInputReader.java b/hudi-bench/src/test/java/org/apache/hudi/bench/reader/TestDFSHoodieDatasetInputReader.java
new file mode 100644
index 0000000..dba57b0
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/reader/TestDFSHoodieDatasetInputReader.java
@@ -0,0 +1,122 @@
+/*
+ * 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.bench.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.HoodieWriteClient;
+import org.apache.hudi.WriteStatus;
+import org.apache.hudi.common.HoodieTestDataGenerator;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTestUtils;
+import org.apache.hudi.common.util.HoodieAvroUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.utilities.UtilitiesTestBase;
+import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestDFSHoodieDatasetInputReader extends UtilitiesTestBase {
+
+  private FilebasedSchemaProvider schemaProvider;
+
+  @BeforeClass
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+  }
+
+  @AfterClass
+  public static void cleanupClass() throws Exception {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @Before
+  public void setup() throws Exception {
+    super.setup();
+    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS("hudi-bench-config/complex-source.avsc"),
+        jsc);
+    HoodieTestUtils.init(jsc.hadoopConfiguration(), dfsBasePath);
+  }
+
+  @After
+  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-bench/src/test/java/org/apache/hudi/bench/utils/TestUtils.java b/hudi-bench/src/test/java/org/apache/hudi/bench/utils/TestUtils.java
new file mode 100644
index 0000000..af2b575
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/utils/TestUtils.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.bench.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.HoodieTestDataGenerator;
+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.avroSchema;
+    for (int i = 0; i < numFiles; i++) {
+      JavaRDD<GenericRecord> rdd = makeRDD(jsc, numRecordsPerFile);
+      AvroConversionUtils.createDataFrame(rdd.rdd(), schema.toString(), sparkSession).write()
+          .format("com.databricks.spark.avro").save(basePath + "/" + i);
+    }
+  }
+
+  public static Schema getSchema() {
+    return HoodieTestDataGenerator.avroSchema;
+  }
+
+}
diff --git a/hudi-bench/src/test/java/org/apache/hudi/bench/writer/TestDeltaWriter.java b/hudi-bench/src/test/java/org/apache/hudi/bench/writer/TestDeltaWriter.java
new file mode 100644
index 0000000..692f552
--- /dev/null
+++ b/hudi-bench/src/test/java/org/apache/hudi/bench/writer/TestDeltaWriter.java
@@ -0,0 +1,114 @@
+/*
+ * 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.bench.writer;
+
+import static junit.framework.TestCase.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.powermock.api.mockito.PowerMockito.when;
+
+import java.util.Properties;
+import org.apache.hudi.HoodieReadClient;
+import org.apache.hudi.HoodieWriteClient;
+import org.apache.hudi.bench.job.HoodieDeltaStreamerWrapper;
+import org.apache.hudi.bench.job.HoodieTestSuiteJob.HoodieTestSuiteConfig;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({DeltaWriter.class})
+public class TestDeltaWriter {
+
+  @Test
+  public void testWorkloadWriterWithDeltaStreamer() throws Exception {
+    JavaSparkContext mockSC = mock(JavaSparkContext.class);
+    HoodieTestSuiteConfig mockCfg = mock(HoodieTestSuiteConfig.class);
+    mockCfg.useDeltaStreamer = true;
+    Properties props = mock(Properties.class);
+    HoodieWriteClient mockWriteClient = PowerMockito.mock(HoodieWriteClient.class);
+    HoodieReadClient mockReadClient = PowerMockito.mock(HoodieReadClient.class);
+    PowerMockito.whenNew(HoodieReadClient.class).withAnyArguments().thenReturn(mockReadClient);
+    PowerMockito.whenNew(HoodieWriteClient.class).withAnyArguments().thenReturn(mockWriteClient);
+    HoodieDeltaStreamerWrapper mockDeltaWrapper = PowerMockito.mock(HoodieDeltaStreamerWrapper.class);
+    PowerMockito.whenNew(HoodieDeltaStreamerWrapper.class).withArguments(mockCfg, mockSC).thenReturn(mockDeltaWrapper);
+
+    String schema = "schema";
+    String checkPoint = "DUMMY";
+    String commitTime = "0";
+    when(mockDeltaWrapper.fetchSource()).thenReturn(Pair.of(null, Pair.of(checkPoint, null)));
+    when(mockDeltaWrapper.insert(Option.empty())).thenReturn(null);
+    when(mockDeltaWrapper.upsert(Option.empty())).thenReturn(null);
+    when(mockDeltaWrapper.bulkInsert(Option.empty())).thenReturn(null);
+
+    when(mockWriteClient.startCommit()).thenReturn(commitTime);
+    DeltaWriter deltaWriter = new DeltaWriter(mockSC, props, mockCfg, schema);
+    assertEquals(deltaWriter.fetchSource().getRight().getKey(), checkPoint);
+    assertEquals(deltaWriter.fetchSource().getRight().getValue(), null);
+    assertFalse(deltaWriter.startCommit().get() == commitTime);
+    assertTrue(deltaWriter.bulkInsert(Option.of(commitTime)) == null);
+    assertTrue(deltaWriter.insert(Option.of(commitTime)) == null);
+    assertTrue(deltaWriter.upsert(Option.of(commitTime)) == null);
+  }
+
+  @Test
+  public void testWorkloadWriterWithHoodieWriteClient() throws Exception {
+    JavaSparkContext mockSC = mock(JavaSparkContext.class);
+    HoodieTestSuiteConfig mockCfg = mock(HoodieTestSuiteConfig.class);
+    mockCfg.useDeltaStreamer = false;
+    mockCfg.targetBasePath = "test";
+    mockCfg.targetTableName = "test";
+    mockCfg.payloadClassName = "test";
+    Properties props = mock(Properties.class);
+    HoodieWriteClient mockWriteClient = PowerMockito.mock(HoodieWriteClient.class);
+    PowerMockito.whenNew(HoodieWriteClient.class).withAnyArguments().thenReturn(mockWriteClient);
+    HoodieReadClient mockReadClient = PowerMockito.mock(HoodieReadClient.class);
+    PowerMockito.whenNew(HoodieReadClient.class).withAnyArguments().thenReturn(mockReadClient);
+    HoodieDeltaStreamerWrapper mockDeltaWrapper = PowerMockito.mock(HoodieDeltaStreamerWrapper.class);
+    PowerMockito.whenNew(HoodieDeltaStreamerWrapper.class).withArguments(mockCfg, mockSC).thenReturn(mockDeltaWrapper);
+
+    String schema = "schema";
+    String checkPoint = "DUMMY";
+    String commitTime = "0";
+
+    when(mockDeltaWrapper.fetchSource()).thenReturn(Pair.of(null, Pair.of(checkPoint, null)));
+    when(mockWriteClient.startCommit()).thenReturn(commitTime);
+    when(mockWriteClient.insert(any(), anyString())).thenReturn(null);
+    when(mockWriteClient.upsert(any(), anyString())).thenReturn(null);
+    when(mockWriteClient.bulkInsert(any(), anyString())).thenReturn(null);
+
+    when(mockWriteClient.startCommit()).thenReturn(commitTime);
+    DeltaWriter deltaWriter = new DeltaWriter(mockSC, props, mockCfg, schema);
+    assertEquals(deltaWriter.fetchSource().getRight().getKey(), checkPoint);
+    assertEquals(deltaWriter.fetchSource().getRight().getValue(), null);
+    assertEquals(deltaWriter.startCommit().get(), commitTime);
+    assertTrue(deltaWriter.bulkInsert(Option.of(commitTime)) == null);
+    assertTrue(deltaWriter.insert(Option.of(commitTime)) == null);
+    assertTrue(deltaWriter.upsert(Option.of(commitTime)) == null);
+  }
+
+}
diff --git a/hudi-bench/src/test/resources/hudi-bench-config/base.properties b/hudi-bench/src/test/resources/hudi-bench-config/base.properties
new file mode 100644
index 0000000..13b1acc
--- /dev/null
+++ b/hudi-bench/src/test/resources/hudi-bench-config/base.properties
@@ -0,0 +1,21 @@
+#
+# 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.upsert.shuffle.parallelism=2
+hoodie.insert.shuffle.parallelism=2
+hoodie.bulkinsert.shuffle.parallelism=2
+hoodie.datasource.write.partitionpath.field=timestamp
diff --git a/hudi-bench/src/test/resources/hudi-bench-config/complex-source.avsc b/hudi-bench/src/test/resources/hudi-bench-config/complex-source.avsc
new file mode 100644
index 0000000..8761306
--- /dev/null
+++ b/hudi-bench/src/test/resources/hudi-bench-config/complex-source.avsc
@@ -0,0 +1,449 @@
+{
+    "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-bench/src/test/resources/hudi-bench-config/complex-workflow-dag-cow.yaml b/hudi-bench/src/test/resources/hudi-bench-config/complex-workflow-dag-cow.yaml
new file mode 100644
index 0000000..3c30c54
--- /dev/null
+++ b/hudi-bench/src/test/resources/hudi-bench-config/complex-workflow-dag-cow.yaml
@@ -0,0 +1,82 @@
+first_insert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    repeat_count: 5
+    num_records_insert: 10000000
+  type: InsertNode
+  deps: none
+second_insert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    repeat_count: 5
+    num_records_insert: 10000000
+  deps: first_insert
+  type: InsertNode
+third_insert:
+  config:
+    record_size: 70000
+    num_insert_partitions: 1
+    repeat_count: 2
+    num_records_insert: 300000
+  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: 300000
+    repeat_count: 5
+    num_records_upsert: 100000
+    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: 300000
+    repeat_count: 5
+    num_records_upsert: 100000
+    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: 22100000
+  type: HiveQueryNode
+  deps: second_upsert
\ No newline at end of file
diff --git a/hudi-bench/src/test/resources/hudi-bench-config/source.avsc b/hudi-bench/src/test/resources/hudi-bench-config/source.avsc
new file mode 100644
index 0000000..4c7892d
--- /dev/null
+++ b/hudi-bench/src/test/resources/hudi-bench-config/source.avsc
@@ -0,0 +1,34 @@
+{
+  "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-bench/src/test/resources/hudi-bench-config/target.avsc b/hudi-bench/src/test/resources/hudi-bench-config/target.avsc
new file mode 100644
index 0000000..d2d4103
--- /dev/null
+++ b/hudi-bench/src/test/resources/hudi-bench-config/target.avsc
@@ -0,0 +1,37 @@
+{
+  "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/hudi-bench/src/test/resources/log4j-surefire-quiet.properties b/hudi-bench/src/test/resources/log4j-surefire-quiet.properties
new file mode 100644
index 0000000..cd7d2db
--- /dev/null
+++ b/hudi-bench/src/test/resources/log4j-surefire-quiet.properties
@@ -0,0 +1,24 @@
+#
+# 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.
+#
+
+log4j.rootLogger=WARN, A1
+# A1 is set to be a ConsoleAppender.
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+# A1 uses PatternLayout.
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
diff --git a/hudi-bench/src/test/resources/log4j-surefire.properties b/hudi-bench/src/test/resources/log4j-surefire.properties
new file mode 100644
index 0000000..1ba3680
--- /dev/null
+++ b/hudi-bench/src/test/resources/log4j-surefire.properties
@@ -0,0 +1,26 @@
+#
+# 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.
+#
+
+log4j.rootLogger=WARN, A1
+log4j.category.org.apache=INFO
+log4j.category.org.apache.parquet.hadoop=WARN
+# A1 is set to be a ConsoleAppender.
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+# A1 uses PatternLayout.
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
diff --git a/hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java b/hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
index 68be7d3..e78dbff 100644
--- a/hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
+++ b/hudi-client/src/test/java/org/apache/hudi/common/HoodieTestDataGenerator.java
@@ -360,6 +360,15 @@ public class HoodieTestDataGenerator {
     });
   }
 
+  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/src/main/java/org/apache/hudi/hive/HiveSyncTool.java b/hudi-hive/src/main/java/org/apache/hudi/hive/HiveSyncTool.java
index 315f575..4cd6a64 100644
--- a/hudi-hive/src/main/java/org/apache/hudi/hive/HiveSyncTool.java
+++ b/hudi-hive/src/main/java/org/apache/hudi/hive/HiveSyncTool.java
@@ -94,6 +94,8 @@ public class HiveSyncTool {
 
     // Check if the necessary table exists
     boolean tableExists = hoodieHiveClient.doesTableExist();
+    // check if the database exists else create it
+    hoodieHiveClient.updateHiveSQL("create database if not exists " + cfg.databaseName);
     // Get the parquet schema for this dataset looking at the latest commit
     MessageType schema = hoodieHiveClient.getDataSchema();
     // Sync schema if needed
diff --git a/hudi-hive/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java b/hudi-hive/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java
index ed319ce..3223108 100644
--- a/hudi-hive/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java
+++ b/hudi-hive/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java
@@ -681,4 +681,8 @@ public class HoodieHiveClient {
       return new PartitionEvent(PartitionEventType.UPDATE, storagePartition);
     }
   }
+
+  public IMetaStoreClient getClient() {
+    return client;
+  }
 }
diff --git a/hudi-hive/src/test/java/org/apache/hudi/hive/util/HiveTestService.java b/hudi-hive/src/test/java/org/apache/hudi/hive/util/HiveTestService.java
index 4ecf280..199bb18 100644
--- a/hudi-hive/src/test/java/org/apache/hudi/hive/util/HiveTestService.java
+++ b/hudi-hive/src/test/java/org/apache/hudi/hive/util/HiveTestService.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hive.metastore.TUGIBasedProcessor;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.thrift.TUGIContainingTransport;
 import org.apache.hive.service.server.HiveServer2;
-import org.apache.hudi.common.model.HoodieTestUtils;
 import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -88,7 +87,7 @@ public class HiveTestService {
     Preconditions.checkState(workDir != null, "The work dir must be set before starting cluster.");
 
     if (hadoopConf == null) {
-      hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
+      hadoopConf = new Configuration();
     }
 
     String localHiveLocation = getHiveLocation(workDir);
diff --git a/hudi-spark/pom.xml b/hudi-spark/pom.xml
index b490dd7..1d2fc29 100644
--- a/hudi-spark/pom.xml
+++ b/hudi-spark/pom.xml
@@ -205,10 +205,22 @@
     <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-core_2.11</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-sql_2.11</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
 
     <!-- Spark (Packages) -->
diff --git a/hudi-spark/src/main/java/org/apache/hudi/ComplexKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/ComplexKeyGenerator.java
index 8419257..f85d248 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/ComplexKeyGenerator.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/ComplexKeyGenerator.java
@@ -19,9 +19,9 @@
 package org.apache.hudi;
 
 import java.util.Arrays;
-import java.util.List;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.common.util.TypedProperties;
 import org.apache.hudi.exception.HoodieException;
 
@@ -30,29 +30,27 @@ import org.apache.hudi.exception.HoodieException;
  */
 public class ComplexKeyGenerator extends KeyGenerator {
 
+  public static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
+  public static final String DEFAULT_RECORD_KEY_SEPARATOR = ":";
   private static final String DEFAULT_PARTITION_PATH = "default";
 
-  private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
-
-  protected final List<String> recordKeyFields;
-
-  protected final List<String> partitionPathFields;
-
   public ComplexKeyGenerator(TypedProperties props) {
     super(props);
     this.recordKeyFields = Arrays.asList(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(","));
-    this.partitionPathFields =
-        Arrays.asList(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(","));
+    this.partitionPathFields = Arrays.asList(props
+        .getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(","));
   }
 
   @Override
   public HoodieKey getKey(GenericRecord record) {
     if (recordKeyFields == null || partitionPathFields == null) {
-      throw new HoodieException("Unable to find field names for record key or partition path in cfg");
+      throw new HoodieException(
+          "Unable to find field names for record key or partition path in cfg");
     }
     StringBuilder recordKey = new StringBuilder();
     for (String recordKeyField : recordKeyFields) {
-      recordKey.append(recordKeyField + ":" + DataSourceUtils.getNestedFieldValAsString(record, recordKeyField) + ",");
+      recordKey.append(StringUtils.join(recordKeyField, DEFAULT_RECORD_KEY_SEPARATOR, DataSourceUtils
+          .getNestedFieldValAsString(record, recordKeyField),","));
     }
     recordKey.deleteCharAt(recordKey.length() - 1);
     StringBuilder partitionPath = new StringBuilder();
@@ -68,12 +66,4 @@ public class ComplexKeyGenerator extends KeyGenerator {
 
     return new HoodieKey(recordKey.toString(), partitionPath.toString());
   }
-
-  public List<String> getRecordKeyFields() {
-    return recordKeyFields;
-  }
-
-  public List<String> getPartitionPathFields() {
-    return partitionPathFields;
-  }
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java
index 9ce79e9..ed97a70 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java
@@ -124,7 +124,7 @@ public class DataSourceUtils {
       throws IOException {
     try {
       return (HoodieRecordPayload) ReflectionUtils.loadClass(payloadClass,
-          new Class<?>[] {GenericRecord.class, Comparable.class}, record, orderingVal);
+          new Class<?>[]{GenericRecord.class, Comparable.class}, record, orderingVal);
     } catch (Throwable e) {
       throw new IOException("Could not create payload for class: " + payloadClass, e);
     }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/EmptyHoodieRecordPayload.java b/hudi-spark/src/main/java/org/apache/hudi/EmptyHoodieRecordPayload.java
index ddcbeb7..f4353a6 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/EmptyHoodieRecordPayload.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/EmptyHoodieRecordPayload.java
@@ -29,7 +29,8 @@ import org.apache.hudi.common.util.Option;
  */
 public class EmptyHoodieRecordPayload implements HoodieRecordPayload<EmptyHoodieRecordPayload> {
 
-  public EmptyHoodieRecordPayload(GenericRecord record, Comparable orderingVal) {}
+  public EmptyHoodieRecordPayload(GenericRecord record, Comparable orderingVal) {
+  }
 
   @Override
   public EmptyHoodieRecordPayload preCombine(EmptyHoodieRecordPayload another) {
diff --git a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
index c35663e..1ca87bb 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
@@ -19,6 +19,7 @@
 package org.apache.hudi;
 
 import java.io.Serializable;
+import java.util.List;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.util.TypedProperties;
@@ -28,6 +29,10 @@ import org.apache.hudi.common.util.TypedProperties;
  */
 public abstract class KeyGenerator implements Serializable {
 
+  protected List<String> recordKeyFields;
+
+  protected List<String> partitionPathFields;
+
   protected transient TypedProperties config;
 
   protected KeyGenerator(TypedProperties config) {
@@ -38,4 +43,12 @@ public abstract class KeyGenerator implements Serializable {
    * Generate a Hoodie Key out of provided generic record.
    */
   public abstract HoodieKey getKey(GenericRecord record);
+
+  public List<String> getRecordKeyFields() {
+    return recordKeyFields;
+  }
+
+  public List<String> getPartitionPathFields() {
+    return partitionPathFields;
+  }
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java
index 50cf327..1489b0c 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/NonpartitionedKeyGenerator.java
@@ -35,7 +35,7 @@ public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
 
   @Override
   public HoodieKey getKey(GenericRecord record) {
-    String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField);
+    String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyFields.get(0));
     return new HoodieKey(recordKey, EMPTY_PARTITION);
   }
 }
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 504ad18..2e40bcc 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java
@@ -44,7 +44,32 @@ import org.apache.hudi.exception.HoodieIOException;
  */
 public class QuickstartUtils {
 
+  private static Option<String> convertToString(HoodieRecord record) {
+    try {
+      String str = HoodieAvroUtils
+          .bytesToAvro(((OverwriteWithLatestAvroPayload) record.getData()).recordBytes, DataGenerator.avroSchema)
+          .toString();
+      str = "{" + str.substring(str.indexOf("\"ts\":"));
+      return Option.of(str.replaceAll("}", ", \"partitionpath\": \"" + record.getPartitionPath() + "\"}"));
+    } catch (IOException e) {
+      return Option.empty();
+    }
+  }
+
+  public static List<String> convertToStringList(List<HoodieRecord> records) {
+    return records.stream().map(hr -> convertToString(hr)).filter(os -> os.isPresent()).map(os -> os.get())
+        .collect(Collectors.toList());
+  }
+
+  public static Map<String, String> getQuickstartWriteConfigs() {
+    Map<String, String> demoConfigs = new HashMap<>();
+    demoConfigs.put("hoodie.insert.shuffle.parallelism", "2");
+    demoConfigs.put("hoodie.upsert.shuffle.parallelism", "2");
+    return demoConfigs;
+  }
+
   public static class DataGenerator {
+
     private static final String DEFAULT_FIRST_PARTITION_PATH = "americas/united_states/san_francisco";
     private static final String DEFAULT_SECOND_PARTITION_PATH = "americas/brazil/sao_paulo";
     private static final String DEFAULT_THIRD_PARTITION_PATH = "asia/india/chennai";
@@ -86,10 +111,6 @@ public class QuickstartUtils {
       return buffer.toString();
     }
 
-    public int getNumExistingKeys() {
-      return numExistingKeys;
-    }
-
     public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName,
         double timestamp) {
       GenericRecord rec = new GenericData.Record(avroSchema);
@@ -118,6 +139,10 @@ public class QuickstartUtils {
       return new OverwriteWithLatestAvroPayload(Option.of(rec));
     }
 
+    public int getNumExistingKeys() {
+      return numExistingKeys;
+    }
+
     /**
      * Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
      */
@@ -171,28 +196,4 @@ public class QuickstartUtils {
       existingKeys.clear();
     }
   }
-
-  private static Option<String> convertToString(HoodieRecord record) {
-    try {
-      String str = HoodieAvroUtils
-          .bytesToAvro(((OverwriteWithLatestAvroPayload) record.getData()).recordBytes, DataGenerator.avroSchema)
-          .toString();
-      str = "{" + str.substring(str.indexOf("\"ts\":"));
-      return Option.of(str.replaceAll("}", ", \"partitionpath\": \"" + record.getPartitionPath() + "\"}"));
-    } catch (IOException e) {
-      return Option.empty();
-    }
-  }
-
-  public static List<String> convertToStringList(List<HoodieRecord> records) {
-    return records.stream().map(hr -> convertToString(hr)).filter(os -> os.isPresent()).map(os -> os.get())
-        .collect(Collectors.toList());
-  }
-
-  public static Map<String, String> getQuickstartWriteConfigs() {
-    Map<String, String> demoConfigs = new HashMap<>();
-    demoConfigs.put("hoodie.insert.shuffle.parallelism", "2");
-    demoConfigs.put("hoodie.upsert.shuffle.parallelism", "2");
-    return demoConfigs;
-  }
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/SimpleKeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/SimpleKeyGenerator.java
index b9bb258..0f32bcd 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/SimpleKeyGenerator.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/SimpleKeyGenerator.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi;
 
+import java.util.Arrays;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.util.TypedProperties;
@@ -30,31 +31,38 @@ public class SimpleKeyGenerator extends KeyGenerator {
 
   private static final String DEFAULT_PARTITION_PATH = "default";
 
-  protected final String recordKeyField;
-
-  protected final String partitionPathField;
-
   public SimpleKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyField = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY());
-    this.partitionPathField = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY());
+    this.recordKeyFields = Arrays.asList(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()));
+    this.partitionPathFields = Arrays.asList(props
+        .getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()));
   }
 
   @Override
   public HoodieKey getKey(GenericRecord record) {
-    if (recordKeyField == null || partitionPathField == null) {
-      throw new HoodieException("Unable to find field names for record key or partition path in cfg");
+    return new HoodieKey(getRecordKey(record), getPartitionPath(record));
+  }
+
+  private String getRecordKey(GenericRecord record) {
+    if (recordKeyFields == null || recordKeyFields.size() < 1) {
+      throw new HoodieException(
+          "Unable to find field names for record key in cfg");
     }
+    return DataSourceUtils.getNestedFieldValAsString(record, recordKeyFields.get(0));
+  }
 
-    String recordKey = DataSourceUtils.getNestedFieldValAsString(record, recordKeyField);
+  private String getPartitionPath(GenericRecord record) {
+    if (partitionPathFields == null || partitionPathFields.size() < 1) {
+      throw new HoodieException(
+          "Unable to find field names for partition path in cfg");
+    }
     String partitionPath;
     try {
-      partitionPath = DataSourceUtils.getNestedFieldValAsString(record, partitionPathField);
+      partitionPath = DataSourceUtils.getNestedFieldValAsString(record, partitionPathFields.get(0));
     } catch (HoodieException e) {
       // if field is not found, lump it into default partition
       partitionPath = DEFAULT_PARTITION_PATH;
     }
-
-    return new HoodieKey(recordKey, partitionPath);
+    return partitionPath;
   }
 }
diff --git a/hudi-spark/src/test/java/TestComplexKeyGenerator.java b/hudi-spark/src/test/java/TestComplexKeyGenerator.java
new file mode 100644
index 0000000..e589c97
--- /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.ComplexKeyGenerator;
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.HoodieTestDataGenerator;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.util.TypedProperties;
+import org.junit.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/src/main/java/org/apache/hudi/utilities/UtilHelpers.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
index bbb18af..9142ee8 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
@@ -27,6 +27,7 @@ import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -43,6 +44,7 @@ import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.index.HoodieIndex;
 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.Transformer;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -220,4 +222,15 @@ public class UtilHelpers {
     defaults.load(in);
     return defaults;
   }
+
+  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-spark/src/main/java/org/apache/hudi/KeyGenerator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/converter/Converter.java
similarity index 59%
copy from hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
copy to hudi-utilities/src/main/java/org/apache/hudi/utilities/converter/Converter.java
index c35663e..13b3d72 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/KeyGenerator.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/converter/Converter.java
@@ -16,26 +16,18 @@
  * limitations under the License.
  */
 
-package org.apache.hudi;
+package org.apache.hudi.utilities.converter;
 
 import java.io.Serializable;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.util.TypedProperties;
+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-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 73ef8a3..94e9e8d 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
@@ -60,6 +60,7 @@ import org.apache.hudi.hive.HiveSyncConfig;
 import org.apache.hudi.hive.HiveSyncTool;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Config;
 import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Operation;
 import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException;
 import org.apache.hudi.utilities.schema.RowBasedSchemaProvider;
@@ -202,8 +203,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();
 
@@ -222,19 +223,19 @@ 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;
   }
 
   /**
    * Read from Upstream Source and apply transformation if needed
    */
-  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();
@@ -285,7 +286,7 @@ public class DeltaSync implements Serializable {
       // default to RowBasedSchemaProvider
       schemaProvider = this.schemaProvider == null || this.schemaProvider.getTargetSchema() == null
           ? transformed.map(r -> (SchemaProvider) new RowBasedSchemaProvider(r.schema())).orElse(
-              dataAndCheckpoint.getSchemaProvider())
+          dataAndCheckpoint.getSchemaProvider())
           : this.schemaProvider;
     } else {
       // Pull the data from the source & prepare the write
@@ -324,7 +325,7 @@ public class DeltaSync implements Serializable {
    * @param metrics Metrics
    * @return Option Compaction instant if one is scheduled
    */
-  private Option<String> writeToSink(JavaRDD<HoodieRecord> records, String checkpointStr,
+  private Pair<Option<String>, JavaRDD<WriteStatus>> writeToSink(JavaRDD<HoodieRecord> records, String checkpointStr,
       HoodieDeltaStreamerMetrics metrics, Timer.Context overallTimerContext) throws Exception {
 
     Option<String> scheduledCompactionInstant = Option.empty();
@@ -380,7 +381,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 {
@@ -405,7 +406,7 @@ public class DeltaSync implements Serializable {
     // Send DeltaStreamer Metrics
     metrics.updateDeltaStreamerMetrics(overallTimeMs, hiveSyncTimeMs);
 
-    return scheduledCompactionInstant;
+    return Pair.of(scheduledCompactionInstant, writeStatusRDD);
   }
 
   private String startCommit() {
@@ -432,16 +433,29 @@ public class DeltaSync implements Serializable {
   /**
    * Sync to Hive
    */
-  private void syncHive() throws ClassNotFoundException {
+  public void syncHiveIfNeeded() throws ClassNotFoundException {
     if (cfg.enableHiveSync) {
+      syncHive();
+    }
+  }
+
+  public void syncHive() {
+    try {
       HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath);
       log.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName + "). Hive metastore URL :"
           + hiveSyncConfig.jdbcUrl + ", basePath :" + cfg.targetBasePath);
-
+      log.info("Hive Conf => " + hiveConf.getAllProperties().toString());
       new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable();
+    } catch (ClassNotFoundException e) {
+      throw new HoodieDeltaStreamerException("Unable to sync with hive", e);
     }
   }
 
+  public void syncHive(HiveConf conf) {
+    this.hiveConf = 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
@@ -514,4 +528,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 db8b40a..3bb91a2 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
@@ -44,6 +44,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hudi.HoodieWriteClient;
 import org.apache.hudi.OverwriteWithLatestAvroPayload;
+import org.apache.hudi.WriteStatus;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.HoodieTimeline;
@@ -62,6 +63,7 @@ import org.apache.hudi.utilities.schema.SchemaProvider;
 import org.apache.hudi.utilities.sources.JsonDFSSource;
 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;
 
@@ -82,7 +84,7 @@ public class HoodieDeltaStreamer implements Serializable {
 
   private 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()),
@@ -390,11 +392,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);
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 4132f5b..0367fe8 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
@@ -19,18 +19,19 @@
 package org.apache.hudi.utilities.deltastreamer;
 
 import com.codahale.metrics.Timer;
+import java.io.Serializable;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.metrics.Metrics;
 
-public class HoodieDeltaStreamerMetrics {
+public class HoodieDeltaStreamerMetrics implements Serializable {
 
   private HoodieWriteConfig config = null;
   private String tableName = null;
 
   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/keygen/TimestampBasedKeyGenerator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/keygen/TimestampBasedKeyGenerator.java
index 970ec54..b8f26f7 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/keygen/TimestampBasedKeyGenerator.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/keygen/TimestampBasedKeyGenerator.java
@@ -78,7 +78,7 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
 
   @Override
   public HoodieKey getKey(GenericRecord record) {
-    Object partitionVal = DataSourceUtils.getNestedFieldVal(record, partitionPathField);
+    Object partitionVal = DataSourceUtils.getNestedFieldVal(record, partitionPathFields.get(0));
     SimpleDateFormat partitionPathFormat = new SimpleDateFormat(outputDateFormat);
     partitionPathFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
 
@@ -97,8 +97,7 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
             "Unexpected type for partition field: " + partitionVal.getClass().getName());
       }
       Date timestamp = this.timestampType == TimestampType.EPOCHMILLISECONDS ? new Date(unixTime) : new Date(unixTime * 1000);
-
-      return new HoodieKey(DataSourceUtils.getNestedFieldValAsString(record, recordKeyField),
+      return new HoodieKey(DataSourceUtils.getNestedFieldValAsString(record, recordKeyFields.get(0)),
           partitionPathFormat.format(timestamp));
     } catch (ParseException pe) {
       throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + partitionVal, pe);
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 b7f6f8c..122156e 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
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.utilities.sources;
 
+import java.io.IOException;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.mapred.AvroKey;
 import org.apache.avro.mapreduce.AvroKeyInputFormat;
@@ -25,6 +26,7 @@ import org.apache.hadoop.io.NullWritable;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.TypedProperties;
 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;
 import org.apache.spark.api.java.JavaPairRDD;
@@ -40,9 +42,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 ce979d6..61579e3 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
@@ -37,21 +37,25 @@ import org.apache.hudi.common.util.TypedProperties;
 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.log4j.LogManager;
+import org.apache.log4j.Logger;
 
 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));
@@ -64,6 +68,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);
@@ -77,7 +82,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/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java
index 5cdb532..99840a4 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieDeltaStreamer.java
@@ -92,13 +92,13 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase {
   @BeforeClass
   public static void initClass() throws Exception {
     UtilitiesTestBase.initClass(true);
-
     // prepare the configs.
-    UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/base.properties", dfs, dfsBasePath + "/base.properties");
-    UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/sql-transformer.properties", dfs,
+    UtilitiesTestBase.Helpers.copyToDFS(Helpers.class.getClassLoader(), "delta-streamer-config/base.properties", dfs,
+        dfsBasePath + "/base.properties");
+    UtilitiesTestBase.Helpers.copyToDFS(Helpers.class.getClassLoader(),"delta-streamer-config/sql-transformer.properties", dfs,
         dfsBasePath + "/sql-transformer.properties");
-    UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc");
-    UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/target.avsc", dfs, dfsBasePath + "/target.avsc");
+    UtilitiesTestBase.Helpers.copyToDFS(Helpers.class.getClassLoader(),"delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc");
+    UtilitiesTestBase.Helpers.copyToDFS(Helpers.class.getClassLoader(),"delta-streamer-config/target.avsc", dfs, dfsBasePath + "/target.avsc");
 
     TypedProperties props = new TypedProperties();
     props.setProperty("include", "sql-transformer.properties");
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/UtilitiesTestBase.java
index 46b0dab..e64e51a 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/UtilitiesTestBase.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/UtilitiesTestBase.java
@@ -157,10 +157,8 @@ public class UtilitiesTestBase {
 
   public static class Helpers {
 
-    // to get hold of resources bundled with jar
-    private static ClassLoader classLoader = Helpers.class.getClassLoader();
-
-    public static void copyToDFS(String testResourcePath, FileSystem fs, String targetPath) throws IOException {
+    public static void copyToDFS(ClassLoader classLoader, String testResourcePath, FileSystem fs, String targetPath)
+        throws IOException {
       BufferedReader reader =
           new BufferedReader(new InputStreamReader(classLoader.getResourceAsStream(testResourcePath)));
       PrintStream os = new PrintStream(fs.create(new Path(targetPath), true));
@@ -195,10 +193,10 @@ public class UtilitiesTestBase {
       }
     }
 
-    public static TypedProperties setupSchemaOnDFS() throws IOException {
-      UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/source.avsc", dfs, dfsBasePath + "/source.avsc");
+    public static TypedProperties setupSchemaOnDFS(String filePath) throws IOException {
+      UtilitiesTestBase.Helpers.copyToDFS(Helpers.class.getClassLoader(), filePath, dfs, dfsBasePath + "/" + filePath);
       TypedProperties props = new TypedProperties();
-      props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
+      props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/" + filePath);
       return props;
     }
 
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDFSSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDFSSource.java
index 4d4fafb..4e1e804 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDFSSource.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestDFSSource.java
@@ -65,7 +65,7 @@ public class TestDFSSource extends UtilitiesTestBase {
   @Before
   public void setup() throws Exception {
     super.setup();
-    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), jsc);
+    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS("delta-streamer-config/source.avsc"), jsc);
   }
 
   @After
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestKafkaSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestKafkaSource.java
index 241fae0..09f8b0b 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestKafkaSource.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestKafkaSource.java
@@ -67,7 +67,7 @@ public class TestKafkaSource extends UtilitiesTestBase {
   @Before
   public void setup() throws Exception {
     super.setup();
-    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS(), jsc);
+    schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFS("delta-streamer-config/source.avsc"), jsc);
     testUtils = new KafkaTestUtils();
     testUtils.setup();
   }
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..8761306
--- /dev/null
+++ b/hudi-utilities/src/test/resources/delta-streamer-config/complex-source.avsc
@@ -0,0 +1,449 @@
+{
+    "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": [
... 794 lines suppressed ...