vinothchandar commented on a change in pull request #1100:
URL: https://github.com/apache/hudi/pull/1100#discussion_r435662207



##########
File path: 
hudi-spark/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java
##########
@@ -38,9 +38,10 @@ public NonpartitionedKeyGenerator(TypedProperties props) {
 
   @Override
   public HoodieKey getKey(GenericRecord record) {
-    String recordKey = DataSourceUtils.getNestedFieldValAsString(record, 
recordKeyField, true);
+    String recordKey = DataSourceUtils.getNestedFieldValAsString(record, 
recordKeyFields.get(0), true);

Review comment:
       Ideally we should make KeyGenerator an interface.. we have it as a class 
so we can force a certain constructor.. so again, this class should be free to 
implement by taking the single key field etc

##########
File path: pom.xml
##########
@@ -549,7 +551,52 @@
         <version>5.17.2</version>
       </dependency>
 
-      <!-- Httpcomponents -->
+      <dependency>

Review comment:
       Why do we need these deps? Conceptually our integ-test Is already 
working with existing bundles and existing dependencies.. we should not need 
any changes to add the test-suite

##########
File path: hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala
##########
@@ -265,6 +265,7 @@ object DataSourceWriteOptions {
   val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = 
"hoodie.datasource.hive_sync.assume_date_partitioning"
   val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = 
"hoodie.datasource.hive_sync.use_pre_apache_input_format"
   val HIVE_USE_JDBC_OPT_KEY = "hoodie.datasource.hive_sync.use_jdbc"
+  val HIVE_ENABLE_TEST_SUITE_OPT_KEY = 
"hoodie.datasource.hive_sync.run_test_suite"

Review comment:
       Please rename/remove

##########
File path: 
hudi-test-suite/src/main/java/org/apache/hudi/testsuite/DeltaWriteClient.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.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.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig;
+import org.apache.hudi.testsuite.dag.nodes.CleanNode;
+import org.apache.hudi.testsuite.dag.nodes.DagNode;
+import org.apache.hudi.testsuite.dag.nodes.RollbackNode;
+import org.apache.hudi.testsuite.dag.nodes.ScheduleCompactNode;
+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 DeltaWriteClient {

Review comment:
       Rename to HudiTestSuiteDriver or anything else like that?

##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/SourceFormatAdapter.java
##########
@@ -71,11 +71,11 @@ public SourceFormatAdapter(Source source) {
                     // 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(),

Review comment:
       Skip this file’s change?

##########
File path: hudi-utilities/pom.xml
##########
@@ -131,6 +131,12 @@
       <artifactId>log4j</artifactId>
     </dependency>
 
+    <dependency>

Review comment:
       Remove this please. We are using log4j directly. This causes issues with 
shading 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
##########
@@ -24,12 +24,17 @@
 import org.apache.avro.generic.GenericRecord;
 
 import java.io.Serializable;
+import java.util.List;
 
 /**
  * Abstract class to extend for plugging in extraction of {@link HoodieKey} 
from an Avro record.
  */
 public abstract class KeyGenerator implements Serializable {
 
+  protected List<String> recordKeyFields;

Review comment:
       This seems to be breaking the abstraction and assuming the key generator 
has to have these fields.. also I don’t understand why we change this for 
purposes of the integration suite. Seems orthogonal to me

##########
File path: hudi-test-suite/README.md
##########
@@ -0,0 +1,300 @@
+<!--
+  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-test-suite` 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
+[INFO] ------------------------------------------------------------------------
+[INFO] Reactor Summary:
+[INFO]
+[INFO] Hudi ............................................... SUCCESS [  2.749 s]
+[INFO] hudi-common ........................................ SUCCESS [ 12.711 s]
+[INFO] hudi-timeline-service .............................. SUCCESS [  1.924 s]
+[INFO] hudi-hadoop-mr ..................................... SUCCESS [  7.203 s]
+[INFO] hudi-client ........................................ SUCCESS [ 10.486 s]
+[INFO] hudi-hive .......................................... SUCCESS [  5.159 s]
+[INFO] hudi-spark ......................................... SUCCESS [ 34.499 s]
+[INFO] hudi-utilities ..................................... SUCCESS [  8.626 s]
+[INFO] hudi-cli ........................................... SUCCESS [ 14.921 s]
+[INFO] hudi-test-suite ......................................... SUCCESS [  
7.706 s]
+[INFO] hudi-hadoop-mr-bundle .............................. SUCCESS [  1.873 s]
+[INFO] hudi-hive-bundle ................................... SUCCESS [  1.508 s]
+[INFO] hudi-spark-bundle .................................. SUCCESS [ 17.432 s]
+[INFO] hudi-presto-bundle ................................. SUCCESS [  1.309 s]
+[INFO] hudi-utilities-bundle .............................. SUCCESS [ 18.386 s]
+[INFO] hudi-timeline-server-bundle ........................ SUCCESS [  8.600 s]
+[INFO] hudi-test-suite-bundle .................................. SUCCESS [ 
38.348 s]
+[INFO] hudi-hadoop-docker ................................. SUCCESS [  2.053 s]
+[INFO] hudi-hadoop-base-docker ............................ SUCCESS [  0.806 s]
+[INFO] hudi-hadoop-namenode-docker ........................ SUCCESS [  0.302 s]
+[INFO] hudi-hadoop-datanode-docker ........................ SUCCESS [  0.403 s]
+[INFO] hudi-hadoop-history-docker ......................... SUCCESS [  0.447 s]
+[INFO] hudi-hadoop-hive-docker ............................ SUCCESS [  1.534 s]
+[INFO] hudi-hadoop-sparkbase-docker ....................... SUCCESS [  0.315 s]
+[INFO] hudi-hadoop-sparkmaster-docker ..................... SUCCESS [  0.407 s]
+[INFO] hudi-hadoop-sparkworker-docker ..................... SUCCESS [  0.447 s]
+[INFO] hudi-hadoop-sparkadhoc-docker ...................... SUCCESS [  0.410 s]
+[INFO] hudi-hadoop-presto-docker .......................... SUCCESS [  0.697 s]
+[INFO] hudi-integ-test .................................... SUCCESS [01:02 min]
+[INFO] ------------------------------------------------------------------------
+[INFO] BUILD SUCCESS
+[INFO] ------------------------------------------------------------------------
+[INFO] Total time: 04:23 min
+[INFO] Finished at: 2019-11-02T23:56:48-07:00
+[INFO] Final Memory: 234M/1582M
+[INFO] ------------------------------------------------------------------------
+```
+
+## 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.testsuite.HoodieTestSuiteJob 
target/hudi-test-suite-0.5.2-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 \ 

Review comment:
       Pull this into a script?

##########
File path: hudi-test-suite/README.md
##########
@@ -0,0 +1,300 @@
+<!--
+  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-test-suite` 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

Review comment:
       Can we skip this output?

##########
File path: hudi-test-suite/prepare_integration_suite.sh
##########
@@ -0,0 +1,130 @@
+#!/bin/bash

Review comment:
       Add license?

##########
File path: 
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",

Review comment:
       Is this same as the schema we saw before? 

##########
File path: pom.xml
##########
@@ -86,7 +88,7 @@
     <junit.vintage.version>5.6.1</junit.vintage.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>

Review comment:
       This is misleading. We have this still in the parent pom? Huh




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to