You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/10/17 15:30:44 UTC

[GitHub] [hudi] nsivabalan commented on a diff in pull request #6888: [HUDI-4982] [DO NOT MERGE] add spark bundle and utilities bundle tests to github actions

nsivabalan commented on code in PR #6888:
URL: https://github.com/apache/hudi/pull/6888#discussion_r997193700


##########
hudi-examples/hudi-examples-spark-slim/pom.xml:
##########
@@ -0,0 +1,298 @@
+<?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-examples</artifactId>
+        <groupId>org.apache.hudi</groupId>
+        <version>0.13.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>hudi-examples-spark-slim</artifactId>
+
+    <properties>
+        <main.basedir>${project.parent.basedir}</main.basedir>
+        <checkstyle.skip>true</checkstyle.skip>
+    </properties>
+
+    <build>
+        <resources>
+            <resource>
+                <directory>src/main/resources</directory>
+            </resource>
+        </resources>
+
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>copy-dependencies</id>
+                        <phase>prepare-package</phase>
+                        <goals>
+                            <goal>copy-dependencies</goal>
+                        </goals>
+                        <configuration>
+                            <outputDirectory>${project.build.directory}/lib</outputDirectory>
+                            <overWriteReleases>true</overWriteReleases>
+                            <overWriteSnapshots>true</overWriteSnapshots>
+                            <overWriteIfNewer>true</overWriteIfNewer>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>net.alchim31.maven</groupId>
+                <artifactId>scala-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>scala-compile-first</id>
+                        <phase>process-resources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                            <goal>compile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <phase>compile</phase>
+                        <goals>
+                            <goal>compile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                        <phase>test-compile</phase>
+                    </execution>
+                </executions>
+                <configuration>
+                    <skip>false</skip>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+
+    <dependencies>
+        <!-- Scala -->
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>
+            <version>${scala.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.scala-lang.modules</groupId>
+            <artifactId>scala-collection-compat_${scala.binary.version}</artifactId>
+        </dependency>
+
+        <!-- Logging -->
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-1.2-api</artifactId>
+        </dependency>
+
+        <!-- Hudi -->
+        <dependency>
+            <groupId>org.apache.hudi</groupId>
+            <artifactId>hudi-examples-common</artifactId>
+            <version>${project.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>*</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hudi</groupId>
+            <artifactId>hudi-client-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+<!--        <dependency>-->
+<!--            <groupId>org.apache.hudi</groupId>-->
+<!--            <artifactId>hudi-java-client</artifactId>-->
+<!--            <version>${project.version}</version>-->
+<!--        </dependency>-->
+
+<!--        <dependency>-->
+<!--            <groupId>org.apache.hudi</groupId>-->
+<!--            <artifactId>hudi-spark-client</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>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hudi</groupId>
+            <artifactId>hudi-spark_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hudi</groupId>
+            <artifactId>hudi-spark-common_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <scope>provided</scope>
+        </dependency>
+
+
+
+
+        <!-- Spark -->
+        <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>
+
+        <!-- Hadoop -->
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-auth</artifactId>
+        </dependency>
+
+        <!-- Parquet -->
+        <dependency>
+            <groupId>org.apache.parquet</groupId>
+            <artifactId>parquet-hadoop</artifactId>
+            <version>${parquet.version}</version>
+        </dependency>
+
+        <!-- Avro -->
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro</artifactId>

Review Comment:
   lets make avro and parquet-avro also provided



##########
hudi-examples/hudi-examples-spark-slim/src/main/java/org/apache/hudi/examples/commonslim/HoodieExampleSparkUtilsSlim.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.examples.commonslim;

Review Comment:
   lets make the package 
   ```
   common.slim
   ```



##########
hudi-examples/hudi-examples-spark-slim/src/main/java/org/apache/hudi/examples/quickstartslim/HoodieSparkQuickstartSlim.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.examples.quickstartslim;
+
+import org.apache.hudi.QuickstartUtils;
+import org.apache.hudi.common.model.HoodieAvroPayload;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.examples.commonslim.HoodieExampleDataGeneratorSlim;
+import org.apache.hudi.examples.commonslim.HoodieExampleSparkUtilsSlim;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
+
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+import java.util.List;
+
+import static org.apache.hudi.config.HoodieWriteConfig.TBL_NAME;
+import static org.apache.spark.sql.SaveMode.Append;
+import static org.apache.spark.sql.SaveMode.Overwrite;
+
+public final class HoodieSparkQuickstartSlim {
+
+  private HoodieSparkQuickstartSlim() {
+  }
+
+  public static void main(String[] args) {
+    if (args.length < 2) {
+      System.err.println("Usage: HoodieWriteClientExample <tablePath> <tableName>");
+      System.exit(1);
+    }
+    String tablePath = args[0];
+    String tableName = args[1];
+
+    SparkSession spark = HoodieExampleSparkUtilsSlim.defaultSparkSession("Hudi Spark basic example");
+    try (JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext())) {
+      runQuickstart(jsc, spark, tableName, tablePath);
+    }
+  }
+
+  /**
+   * Visible for testing
+   */
+  public static void runQuickstart(JavaSparkContext jsc, SparkSession spark, String tableName, String tablePath) {

Review Comment:
   we have duplicated the code b/w hudi-examples-spark and hudi-examples-spark-slim. can you create a follow up jira on this. we should try to dedup later.



##########
hudi-examples/hudi-examples-spark-slim/src/main/java/org/apache/hudi/examples/quickstartslim/HoodieSparkQuickstartSlim.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.examples.quickstartslim;

Review Comment:
   similar suggestion as above



##########
scripts/test_data/ny.avsc:
##########
@@ -0,0 +1,82 @@
+{

Review Comment:
   lets see if we re-use the stocks data we have. 
   data is located at HUID_REPO/docker/demo/data/batch_1.json
   
   



##########
scripts/run_spark_bundle_test.sh:
##########
@@ -0,0 +1,47 @@
+#!/bin/bash
+
+#  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.
+
+source ${PWD}/scripts/get_spark_hadoop.sh
+process_spark_version
+process_scala_version
+
+VERSION_STRING="${SCALA_NUM}_${SPARK_PROFILE}"
+
+#Set spark env vars
+export SPARK_HOME=${PWD}/${SPARK_FOLDER_NAME}
+export PATH=${PATH}:${SPARK_HOME}/bin:${SPARK_HOME}/sbin
+
+#download and extract file
+if [[ ! -d $SPARK_HOME ]]; then
+    download_spark
+fi 
+
+
+
+SPARK_BUNDLE_NAME=${PWD}/packaging/hudi-spark-bundle/target/hudi-${SPARK_PROFILE}-bundle_${SCALA_NUM}-0.13.0-SNAPSHOT.jar
+SLIM_EXAMPLE_BUNDLE=${PWD}/hudi-examples/hudi-examples-spark-slim/target/hudi-examples-spark-slim-0.13.0-SNAPSHOT.jar
+
+SPARK_SUBMIT=${SPARK_HOME}/bin/spark-submit
+OUTPUT_FOLDER_NAME="/tmp/spark-bundle-test_${VERSION_STRING}/"

Review Comment:
   can you ensure that multiple profiles won't collide. for eg, w/ diff flink profile versions. 



##########
.github/workflows/bot.yml:
##########
@@ -57,18 +57,20 @@ jobs:
           FLINK_PROFILE: ${{ matrix.flinkProfile }}
         run:
           mvn clean install -Pintegration-tests -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -D"$FLINK_PROFILE" -DskipTests=true $MVN_ARGS
-      - name: Quickstart Test
+      - name: Utilities Bundle Tests
         env:
           SCALA_PROFILE: ${{ matrix.scalaProfile }}
           SPARK_PROFILE: ${{ matrix.sparkProfile }}
           FLINK_PROFILE: ${{ matrix.flinkProfile }}
+        if: ${{ !endsWith(env.FLINK_PROFILE, '1.13') }} # skip test flink1.13 tests as changing flink version doesn't matter for this
         run:
-          mvn test -Punit-tests -D"$SCALA_PROFILE" -D"$SPARK_PROFILE" -D"$FLINK_PROFILE" -DfailIfNoTests=false -pl hudi-examples/hudi-examples-flink,hudi-examples/hudi-examples-java,hudi-examples/hudi-examples-spark $MVN_ARGS
-      - name: Spark SQL Test
+          ./scripts/run_utilities_bundle_test.sh && ./scripts/run_utilities_bundle_test.sh slim

Review Comment:
   I assume you will fix this before final merge.



##########
hudi-examples/hudi-examples-spark-slim/pom.xml:
##########
@@ -0,0 +1,298 @@
+<?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-examples</artifactId>
+        <groupId>org.apache.hudi</groupId>
+        <version>0.13.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>hudi-examples-spark-slim</artifactId>
+
+    <properties>
+        <main.basedir>${project.parent.basedir}</main.basedir>
+        <checkstyle.skip>true</checkstyle.skip>
+    </properties>
+
+    <build>
+        <resources>
+            <resource>
+                <directory>src/main/resources</directory>
+            </resource>
+        </resources>
+
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>copy-dependencies</id>
+                        <phase>prepare-package</phase>
+                        <goals>
+                            <goal>copy-dependencies</goal>
+                        </goals>
+                        <configuration>
+                            <outputDirectory>${project.build.directory}/lib</outputDirectory>
+                            <overWriteReleases>true</overWriteReleases>
+                            <overWriteSnapshots>true</overWriteSnapshots>
+                            <overWriteIfNewer>true</overWriteIfNewer>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>net.alchim31.maven</groupId>
+                <artifactId>scala-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>scala-compile-first</id>
+                        <phase>process-resources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                            <goal>compile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <phase>compile</phase>
+                        <goals>
+                            <goal>compile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                        <phase>test-compile</phase>
+                    </execution>
+                </executions>
+                <configuration>
+                    <skip>false</skip>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+            </plugin>
+        </plugins>
+    </build>
+
+    <dependencies>
+        <!-- Scala -->
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>
+            <version>${scala.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.scala-lang.modules</groupId>
+            <artifactId>scala-collection-compat_${scala.binary.version}</artifactId>
+        </dependency>
+
+        <!-- Logging -->
+        <dependency>
+            <groupId>org.apache.logging.log4j</groupId>
+            <artifactId>log4j-1.2-api</artifactId>
+        </dependency>
+
+        <!-- Hudi -->
+        <dependency>
+            <groupId>org.apache.hudi</groupId>
+            <artifactId>hudi-examples-common</artifactId>
+            <version>${project.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>*</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hudi</groupId>
+            <artifactId>hudi-client-common</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+<!--        <dependency>-->

Review Comment:
   can we remove the commented out lines



##########
hudi-examples/hudi-examples-spark-slim/src/main/java/org/apache/hudi/examples/commonslim/HoodieExampleSparkUtilsSlim.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.examples.commonslim;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.sql.SparkSession;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Bunch of util methods.
+ */
+public class HoodieExampleSparkUtilsSlim {

Review Comment:
   we don't need to suffix slim in the end. anyways, this is in a diff package (common.slim)



##########
scripts/test_data/parquet-dfs-compact.props:
##########
@@ -0,0 +1,8 @@
+hoodie.datasource.write.recordkey.field=VendorID
+hoodie.datasource.write.partitionpath.field=date_col

Review Comment:
   lets name the file properly. utilities-bundle-test.props or something.



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java:
##########
@@ -437,92 +437,92 @@ public boolean equals(Object o) {
       }
       Config config = (Config) o;
       return sourceLimit == config.sourceLimit
-              && Objects.equals(targetBasePath, config.targetBasePath)
-              && Objects.equals(targetTableName, config.targetTableName)
-              && Objects.equals(tableType, config.tableType)
-              && Objects.equals(baseFileFormat, config.baseFileFormat)
-              && Objects.equals(propsFilePath, config.propsFilePath)
-              && Objects.equals(configs, config.configs)
-              && Objects.equals(sourceClassName, config.sourceClassName)
-              && Objects.equals(sourceOrderingField, config.sourceOrderingField)
-              && Objects.equals(payloadClassName, config.payloadClassName)
-              && Objects.equals(schemaProviderClassName, config.schemaProviderClassName)
-              && Objects.equals(transformerClassNames, config.transformerClassNames)
-              && operation == config.operation
-              && Objects.equals(filterDupes, config.filterDupes)
-              && Objects.equals(enableHiveSync, config.enableHiveSync)
-              && Objects.equals(enableMetaSync, config.enableMetaSync)
-              && Objects.equals(forceEmptyMetaSync, config.forceEmptyMetaSync)
-              && Objects.equals(syncClientToolClassNames, config.syncClientToolClassNames)
-              && Objects.equals(maxPendingCompactions, config.maxPendingCompactions)
-              && Objects.equals(maxPendingClustering, config.maxPendingClustering)
-              && Objects.equals(continuousMode, config.continuousMode)
-              && Objects.equals(minSyncIntervalSeconds, config.minSyncIntervalSeconds)
-              && Objects.equals(sparkMaster, config.sparkMaster)
-              && Objects.equals(commitOnErrors, config.commitOnErrors)
-              && Objects.equals(deltaSyncSchedulingWeight, config.deltaSyncSchedulingWeight)
-              && Objects.equals(compactSchedulingWeight, config.compactSchedulingWeight)
-              && Objects.equals(clusterSchedulingWeight, config.clusterSchedulingWeight)
-              && Objects.equals(deltaSyncSchedulingMinShare, config.deltaSyncSchedulingMinShare)
-              && Objects.equals(compactSchedulingMinShare, config.compactSchedulingMinShare)
-              && Objects.equals(clusterSchedulingMinShare, config.clusterSchedulingMinShare)
-              && Objects.equals(forceDisableCompaction, config.forceDisableCompaction)
-              && Objects.equals(checkpoint, config.checkpoint)
-              && Objects.equals(initialCheckpointProvider, config.initialCheckpointProvider)
-              && Objects.equals(help, config.help);
+          && Objects.equals(targetBasePath, config.targetBasePath)
+          && Objects.equals(targetTableName, config.targetTableName)
+          && Objects.equals(tableType, config.tableType)
+          && Objects.equals(baseFileFormat, config.baseFileFormat)
+          && Objects.equals(propsFilePath, config.propsFilePath)
+          && Objects.equals(configs, config.configs)
+          && Objects.equals(sourceClassName, config.sourceClassName)
+          && Objects.equals(sourceOrderingField, config.sourceOrderingField)
+          && Objects.equals(payloadClassName, config.payloadClassName)
+          && Objects.equals(schemaProviderClassName, config.schemaProviderClassName)
+          && Objects.equals(transformerClassNames, config.transformerClassNames)
+          && operation == config.operation
+          && Objects.equals(filterDupes, config.filterDupes)
+          && Objects.equals(enableHiveSync, config.enableHiveSync)
+          && Objects.equals(enableMetaSync, config.enableMetaSync)
+          && Objects.equals(forceEmptyMetaSync, config.forceEmptyMetaSync)
+          && Objects.equals(syncClientToolClassNames, config.syncClientToolClassNames)
+          && Objects.equals(maxPendingCompactions, config.maxPendingCompactions)
+          && Objects.equals(maxPendingClustering, config.maxPendingClustering)
+          && Objects.equals(continuousMode, config.continuousMode)
+          && Objects.equals(minSyncIntervalSeconds, config.minSyncIntervalSeconds)
+          && Objects.equals(sparkMaster, config.sparkMaster)
+          && Objects.equals(commitOnErrors, config.commitOnErrors)
+          && Objects.equals(deltaSyncSchedulingWeight, config.deltaSyncSchedulingWeight)
+          && Objects.equals(compactSchedulingWeight, config.compactSchedulingWeight)
+          && Objects.equals(clusterSchedulingWeight, config.clusterSchedulingWeight)
+          && Objects.equals(deltaSyncSchedulingMinShare, config.deltaSyncSchedulingMinShare)
+          && Objects.equals(compactSchedulingMinShare, config.compactSchedulingMinShare)
+          && Objects.equals(clusterSchedulingMinShare, config.clusterSchedulingMinShare)
+          && Objects.equals(forceDisableCompaction, config.forceDisableCompaction)
+          && Objects.equals(checkpoint, config.checkpoint)
+          && Objects.equals(initialCheckpointProvider, config.initialCheckpointProvider)
+          && Objects.equals(help, config.help);

Review Comment:
   undo unintentional changes please



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java:
##########
@@ -653,8 +653,8 @@ public DeltaSyncService(Config cfg, JavaSparkContext jssc, FileSystem fs, Config
                 + cfg.baseFileFormat);
         cfg.baseFileFormat = baseFileFormat;
         this.cfg.baseFileFormat = baseFileFormat;
-        Map<String,String> propsToValidate = new HashMap<>();
-        properties.get().forEach((k,v) -> propsToValidate.put(k.toString(),v.toString()));
+        Map<String, String> propsToValidate = new HashMap<>();
+        properties.get().forEach((k, v) -> propsToValidate.put(k.toString(), v.toString()));

Review Comment:
   same comment for this entire class



##########
scripts/run_spark_bundle_test.sh:
##########
@@ -0,0 +1,47 @@
+#!/bin/bash
+
+#  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.
+
+source ${PWD}/scripts/get_spark_hadoop.sh
+process_spark_version
+process_scala_version
+
+VERSION_STRING="${SCALA_NUM}_${SPARK_PROFILE}"

Review Comment:
   even scala, its the version. lets name it "SCALA_VERSION" or "SCALA_PROFILE"



##########
scripts/run_spark_bundle_test.sh:
##########
@@ -0,0 +1,47 @@
+#!/bin/bash
+
+#  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.
+
+source ${PWD}/scripts/get_spark_hadoop.sh
+process_spark_version
+process_scala_version
+
+VERSION_STRING="${SCALA_NUM}_${SPARK_PROFILE}"
+
+#Set spark env vars
+export SPARK_HOME=${PWD}/${SPARK_FOLDER_NAME}
+export PATH=${PATH}:${SPARK_HOME}/bin:${SPARK_HOME}/sbin
+
+#download and extract file
+if [[ ! -d $SPARK_HOME ]]; then
+    download_spark
+fi 
+
+
+
+SPARK_BUNDLE_NAME=${PWD}/packaging/hudi-spark-bundle/target/hudi-${SPARK_PROFILE}-bundle_${SCALA_NUM}-0.13.0-SNAPSHOT.jar
+SLIM_EXAMPLE_BUNDLE=${PWD}/hudi-examples/hudi-examples-spark-slim/target/hudi-examples-spark-slim-0.13.0-SNAPSHOT.jar
+
+SPARK_SUBMIT=${SPARK_HOME}/bin/spark-submit
+OUTPUT_FOLDER_NAME="/tmp/spark-bundle-test_${VERSION_STRING}/"
+SPARK_SUBMIT_COMMAND="${SPARK_SUBMIT} --driver-memory 8g --executor-memory 8g"
+SPARK_SUBMIT_COMMAND+=" --class org.apache.hudi.examples.quickstartslim.HoodieSparkQuickstartSlim"

Review Comment:
   we don't really need to do += for every line. it can be just a single string. 



##########
scripts/run_utilities_bundle_test.sh:
##########
@@ -0,0 +1,114 @@
+#!/bin/bash
+
+#  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.
+
+source ${PWD}/scripts/get_spark_hadoop.sh
+process_spark_version
+process_scala_version
+
+VERSION_STRING="${SCALA_NUM}_${SPARK_PROFILE}"
+
+#Set spark env vars
+export SPARK_HOME=${PWD}/${SPARK_FOLDER_NAME}
+export PATH=${PATH}:${SPARK_HOME}/bin:${SPARK_HOME}/sbin
+
+
+#download and extract file
+if [[ ! -d $SPARK_HOME ]]; then
+    download_spark
+fi 
+
+#Create props file
+TEST_DATA_FOLDER=${PWD}/scripts/test_data
+PROPS_FILE_TEMPLATE=${TEST_DATA_FOLDER}/parquet-dfs-compact.props
+PROPS_FILE="${TEST_DATA_FOLDER}/parquet-dfs-compact_${VERSION_STRING}.props"
+rm -rf $PROPS_FILE
+touch $PROPS_FILE
+cat $PROPS_FILE_TEMPLATE >> $PROPS_FILE
+PARQUET_SOURCE="${TEST_DATA_FOLDER}/parquet_src"
+echo hoodie.deltastreamer.source.dfs.root=${PARQUET_SOURCE} >> $PROPS_FILE
+echo hoodie.deltastreamer.schemaprovider.target.schema.file=file:${TEST_DATA_FOLDER}/ny.avsc >> $PROPS_FILE
+echo hoodie.deltastreamer.schemaprovider.source.schema.file=file:${TEST_DATA_FOLDER}/ny.avsc >> $PROPS_FILE
+
+#Create spark submit command
+SPARK_SUBMIT=${SPARK_HOME}/bin/spark-submit
+OUTPUT_FOLDER_NAME="/tmp/hudi-deltastreamer-ny_${VERSION_STRING}/"
+UTILITIES_BUNDLE_NAME=${PWD}/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_${SCALA_NUM}-0.13.0-SNAPSHOT.jar
+UTILITIES_SLIM_BUNDLE_NAME=${PWD}/packaging/hudi-utilities-slim-bundle/target/hudi-utilities-slim-bundle_${SCALA_NUM}-0.13.0-SNAPSHOT.jar
+SPARK_BUNDLE_NAME=${PWD}/packaging/hudi-spark-bundle/target/hudi-${SPARK_PROFILE}-bundle_${SCALA_NUM}-0.13.0-SNAPSHOT.jar
+BUNDLE_TEST_TYPE=$1
+JARS_ARG="${SPARK_BUNDLE_NAME} ${UTILITIES_SLIM_BUNDLE_NAME}"
+JARS_BEGIN="--jars"
+SLIM_BUNDLE="true"
+if [ "$BUNDLE_TEST_TYPE" != "slim" ]; then
+    JARS_ARG="$UTILITIES_BUNDLE_NAME"
+    JARS_BEGIN=""
+    SLIM_BUNDLE="false"
+fi
+
+
+SPARK_SUBMIT_COMMAND="${SPARK_SUBMIT} --driver-memory 8g --executor-memory 8g"
+SPARK_SUBMIT_COMMAND+=" --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer"
+SPARK_SUBMIT_COMMAND+=" ${JARS_BEGIN} ${JARS_ARG} --props ${PROPS_FILE}"
+SPARK_SUBMIT_COMMAND+=" --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider"
+SPARK_SUBMIT_COMMAND+=" --source-class org.apache.hudi.utilities.sources.ParquetDFSSource"
+SPARK_SUBMIT_COMMAND+=" --source-ordering-field date_col --table-type MERGE_ON_READ"
+SPARK_SUBMIT_COMMAND+=" --target-base-path file://${OUTPUT_FOLDER_NAME}"
+SPARK_SUBMIT_COMMAND+=" --target-table ny_hudi_tbl  --op UPSERT"
+
+echo "::warning::run_utilities_bundle_test.sh running spark submit command (slim bundle?: ${SLIM_BUNDLE})"
+#run spark submit command
+rm -rf $OUTPUT_FOLDER_NAME
+$SPARK_SUBMIT_COMMAND || { echo "::error::run_utilities_bundle_test.sh deltastreamer failed (slim bundle?: ${SLIM_BUNDLE})" && exit 1; }
+
+#make sure that the output folder has a bunch of data in it
+OUTPUT_SIZE=$(du -s ${OUTPUT_FOLDER_NAME} | awk '{print $1}')
+echo "::warning::run_utilities_bundle_test.sh done with spark submit, output size is ${OUTPUT_SIZE} (slim bundle?: ${SLIM_BUNDLE})"
+if [[ -z $OUTPUT_SIZE || "$OUTPUT_SIZE" -lt "1000" ]]; then
+    echo "::error::run_utilities_bundle_test.sh deltastreamer output folder is much smaller than expected (slim bundle?: ${SLIM_BUNDLE})" 
+    exit 1
+fi
+
+#create scala commands file
+COMMANDS_FILE="$TEST_DATA_FOLDER/commands_${VERSION_STRING}.scala"
+rm -rf $COMMANDS_FILE
+touch $COMMANDS_FILE
+echo "val hudiDf = spark.read.format(\"org.apache.hudi\").load(\"${OUTPUT_FOLDER_NAME}\")" >> $COMMANDS_FILE
+echo "val df = spark.read.format(\"parquet\").load(\"${PARQUET_SOURCE}\")" >> $COMMANDS_FILE
+cat $TEST_DATA_FOLDER/commands.scala >> $COMMANDS_FILE
+
+#create spark shell command
+SPARK_SHELL=${SPARK_HOME}/bin/spark-shell
+SPARK_SHELL_COMMAND="${SPARK_SHELL} --jars ${JARS_ARG}" 
+SPARK_SHELL_COMMAND+=" --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer'"

Review Comment:
   can we name this VALIDATION_COMMAND instead of "SPARK_SHELL_COMMAND"



##########
scripts/run_utilities_bundle_test.sh:
##########
@@ -0,0 +1,114 @@
+#!/bin/bash
+
+#  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.
+
+source ${PWD}/scripts/get_spark_hadoop.sh
+process_spark_version
+process_scala_version
+
+VERSION_STRING="${SCALA_NUM}_${SPARK_PROFILE}"
+
+#Set spark env vars
+export SPARK_HOME=${PWD}/${SPARK_FOLDER_NAME}
+export PATH=${PATH}:${SPARK_HOME}/bin:${SPARK_HOME}/sbin
+
+
+#download and extract file
+if [[ ! -d $SPARK_HOME ]]; then
+    download_spark
+fi 
+
+#Create props file
+TEST_DATA_FOLDER=${PWD}/scripts/test_data
+PROPS_FILE_TEMPLATE=${TEST_DATA_FOLDER}/parquet-dfs-compact.props
+PROPS_FILE="${TEST_DATA_FOLDER}/parquet-dfs-compact_${VERSION_STRING}.props"
+rm -rf $PROPS_FILE
+touch $PROPS_FILE
+cat $PROPS_FILE_TEMPLATE >> $PROPS_FILE
+PARQUET_SOURCE="${TEST_DATA_FOLDER}/parquet_src"
+echo hoodie.deltastreamer.source.dfs.root=${PARQUET_SOURCE} >> $PROPS_FILE
+echo hoodie.deltastreamer.schemaprovider.target.schema.file=file:${TEST_DATA_FOLDER}/ny.avsc >> $PROPS_FILE
+echo hoodie.deltastreamer.schemaprovider.source.schema.file=file:${TEST_DATA_FOLDER}/ny.avsc >> $PROPS_FILE
+
+#Create spark submit command
+SPARK_SUBMIT=${SPARK_HOME}/bin/spark-submit
+OUTPUT_FOLDER_NAME="/tmp/hudi-deltastreamer-ny_${VERSION_STRING}/"
+UTILITIES_BUNDLE_NAME=${PWD}/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_${SCALA_NUM}-0.13.0-SNAPSHOT.jar
+UTILITIES_SLIM_BUNDLE_NAME=${PWD}/packaging/hudi-utilities-slim-bundle/target/hudi-utilities-slim-bundle_${SCALA_NUM}-0.13.0-SNAPSHOT.jar
+SPARK_BUNDLE_NAME=${PWD}/packaging/hudi-spark-bundle/target/hudi-${SPARK_PROFILE}-bundle_${SCALA_NUM}-0.13.0-SNAPSHOT.jar
+BUNDLE_TEST_TYPE=$1
+JARS_ARG="${SPARK_BUNDLE_NAME} ${UTILITIES_SLIM_BUNDLE_NAME}"
+JARS_BEGIN="--jars"
+SLIM_BUNDLE="true"
+if [ "$BUNDLE_TEST_TYPE" != "slim" ]; then
+    JARS_ARG="$UTILITIES_BUNDLE_NAME"
+    JARS_BEGIN=""
+    SLIM_BUNDLE="false"
+fi
+
+
+SPARK_SUBMIT_COMMAND="${SPARK_SUBMIT} --driver-memory 8g --executor-memory 8g"
+SPARK_SUBMIT_COMMAND+=" --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer"
+SPARK_SUBMIT_COMMAND+=" ${JARS_BEGIN} ${JARS_ARG} --props ${PROPS_FILE}"
+SPARK_SUBMIT_COMMAND+=" --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider"
+SPARK_SUBMIT_COMMAND+=" --source-class org.apache.hudi.utilities.sources.ParquetDFSSource"
+SPARK_SUBMIT_COMMAND+=" --source-ordering-field date_col --table-type MERGE_ON_READ"
+SPARK_SUBMIT_COMMAND+=" --target-base-path file://${OUTPUT_FOLDER_NAME}"
+SPARK_SUBMIT_COMMAND+=" --target-table ny_hudi_tbl  --op UPSERT"
+
+echo "::warning::run_utilities_bundle_test.sh running spark submit command (slim bundle?: ${SLIM_BUNDLE})"
+#run spark submit command
+rm -rf $OUTPUT_FOLDER_NAME
+$SPARK_SUBMIT_COMMAND || { echo "::error::run_utilities_bundle_test.sh deltastreamer failed (slim bundle?: ${SLIM_BUNDLE})" && exit 1; }
+
+#make sure that the output folder has a bunch of data in it
+OUTPUT_SIZE=$(du -s ${OUTPUT_FOLDER_NAME} | awk '{print $1}')
+echo "::warning::run_utilities_bundle_test.sh done with spark submit, output size is ${OUTPUT_SIZE} (slim bundle?: ${SLIM_BUNDLE})"
+if [[ -z $OUTPUT_SIZE || "$OUTPUT_SIZE" -lt "1000" ]]; then
+    echo "::error::run_utilities_bundle_test.sh deltastreamer output folder is much smaller than expected (slim bundle?: ${SLIM_BUNDLE})" 
+    exit 1
+fi
+
+#create scala commands file
+COMMANDS_FILE="$TEST_DATA_FOLDER/commands_${VERSION_STRING}.scala"
+rm -rf $COMMANDS_FILE
+touch $COMMANDS_FILE
+echo "val hudiDf = spark.read.format(\"org.apache.hudi\").load(\"${OUTPUT_FOLDER_NAME}\")" >> $COMMANDS_FILE
+echo "val df = spark.read.format(\"parquet\").load(\"${PARQUET_SOURCE}\")" >> $COMMANDS_FILE

Review Comment:
   df -> inputDf



##########
scripts/run_utilities_bundle_test.sh:
##########
@@ -0,0 +1,114 @@
+#!/bin/bash
+
+#  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.
+
+source ${PWD}/scripts/get_spark_hadoop.sh
+process_spark_version
+process_scala_version
+
+VERSION_STRING="${SCALA_NUM}_${SPARK_PROFILE}"
+
+#Set spark env vars
+export SPARK_HOME=${PWD}/${SPARK_FOLDER_NAME}
+export PATH=${PATH}:${SPARK_HOME}/bin:${SPARK_HOME}/sbin
+
+
+#download and extract file
+if [[ ! -d $SPARK_HOME ]]; then
+    download_spark
+fi 
+
+#Create props file
+TEST_DATA_FOLDER=${PWD}/scripts/test_data
+PROPS_FILE_TEMPLATE=${TEST_DATA_FOLDER}/parquet-dfs-compact.props
+PROPS_FILE="${TEST_DATA_FOLDER}/parquet-dfs-compact_${VERSION_STRING}.props"
+rm -rf $PROPS_FILE
+touch $PROPS_FILE
+cat $PROPS_FILE_TEMPLATE >> $PROPS_FILE
+PARQUET_SOURCE="${TEST_DATA_FOLDER}/parquet_src"
+echo hoodie.deltastreamer.source.dfs.root=${PARQUET_SOURCE} >> $PROPS_FILE
+echo hoodie.deltastreamer.schemaprovider.target.schema.file=file:${TEST_DATA_FOLDER}/ny.avsc >> $PROPS_FILE
+echo hoodie.deltastreamer.schemaprovider.source.schema.file=file:${TEST_DATA_FOLDER}/ny.avsc >> $PROPS_FILE
+
+#Create spark submit command
+SPARK_SUBMIT=${SPARK_HOME}/bin/spark-submit
+OUTPUT_FOLDER_NAME="/tmp/hudi-deltastreamer-ny_${VERSION_STRING}/"
+UTILITIES_BUNDLE_NAME=${PWD}/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_${SCALA_NUM}-0.13.0-SNAPSHOT.jar
+UTILITIES_SLIM_BUNDLE_NAME=${PWD}/packaging/hudi-utilities-slim-bundle/target/hudi-utilities-slim-bundle_${SCALA_NUM}-0.13.0-SNAPSHOT.jar
+SPARK_BUNDLE_NAME=${PWD}/packaging/hudi-spark-bundle/target/hudi-${SPARK_PROFILE}-bundle_${SCALA_NUM}-0.13.0-SNAPSHOT.jar
+BUNDLE_TEST_TYPE=$1
+JARS_ARG="${SPARK_BUNDLE_NAME} ${UTILITIES_SLIM_BUNDLE_NAME}"
+JARS_BEGIN="--jars"
+SLIM_BUNDLE="true"
+if [ "$BUNDLE_TEST_TYPE" != "slim" ]; then
+    JARS_ARG="$UTILITIES_BUNDLE_NAME"
+    JARS_BEGIN=""
+    SLIM_BUNDLE="false"
+fi
+
+
+SPARK_SUBMIT_COMMAND="${SPARK_SUBMIT} --driver-memory 8g --executor-memory 8g"
+SPARK_SUBMIT_COMMAND+=" --class org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer"
+SPARK_SUBMIT_COMMAND+=" ${JARS_BEGIN} ${JARS_ARG} --props ${PROPS_FILE}"
+SPARK_SUBMIT_COMMAND+=" --schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider"
+SPARK_SUBMIT_COMMAND+=" --source-class org.apache.hudi.utilities.sources.ParquetDFSSource"
+SPARK_SUBMIT_COMMAND+=" --source-ordering-field date_col --table-type MERGE_ON_READ"
+SPARK_SUBMIT_COMMAND+=" --target-base-path file://${OUTPUT_FOLDER_NAME}"
+SPARK_SUBMIT_COMMAND+=" --target-table ny_hudi_tbl  --op UPSERT"
+
+echo "::warning::run_utilities_bundle_test.sh running spark submit command (slim bundle?: ${SLIM_BUNDLE})"
+#run spark submit command
+rm -rf $OUTPUT_FOLDER_NAME
+$SPARK_SUBMIT_COMMAND || { echo "::error::run_utilities_bundle_test.sh deltastreamer failed (slim bundle?: ${SLIM_BUNDLE})" && exit 1; }
+
+#make sure that the output folder has a bunch of data in it
+OUTPUT_SIZE=$(du -s ${OUTPUT_FOLDER_NAME} | awk '{print $1}')
+echo "::warning::run_utilities_bundle_test.sh done with spark submit, output size is ${OUTPUT_SIZE} (slim bundle?: ${SLIM_BUNDLE})"
+if [[ -z $OUTPUT_SIZE || "$OUTPUT_SIZE" -lt "1000" ]]; then
+    echo "::error::run_utilities_bundle_test.sh deltastreamer output folder is much smaller than expected (slim bundle?: ${SLIM_BUNDLE})" 
+    exit 1
+fi
+
+#create scala commands file
+COMMANDS_FILE="$TEST_DATA_FOLDER/commands_${VERSION_STRING}.scala"
+rm -rf $COMMANDS_FILE
+touch $COMMANDS_FILE
+echo "val hudiDf = spark.read.format(\"org.apache.hudi\").load(\"${OUTPUT_FOLDER_NAME}\")" >> $COMMANDS_FILE
+echo "val df = spark.read.format(\"parquet\").load(\"${PARQUET_SOURCE}\")" >> $COMMANDS_FILE
+cat $TEST_DATA_FOLDER/commands.scala >> $COMMANDS_FILE
+
+#create spark shell command
+SPARK_SHELL=${SPARK_HOME}/bin/spark-shell
+SPARK_SHELL_COMMAND="${SPARK_SHELL} --jars ${JARS_ARG}" 
+SPARK_SHELL_COMMAND+=" --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer'"

Review Comment:
   and is it possible to reuse validation across spark bundle test and utilities bundle tests.



##########
hudi-examples/hudi-examples-spark-slim/src/main/java/org/apache/hudi/examples/commonslim/HoodieExampleSparkUtilsSlim.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.examples.commonslim;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.sql.SparkSession;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Bunch of util methods.
+ */
+public class HoodieExampleSparkUtilsSlim {

Review Comment:
   similarly, lets fix all class names. 



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

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