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