nsivabalan commented on a change in pull request #2747:
URL: https://github.com/apache/hudi/pull/2747#discussion_r646499620



##########
File path: 
hudi-utilities/src/test/java/org/apache/hudi/utilities/transform/TestSqlFileBasedTransformer.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.utilities.transform;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.parser.ParseException;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+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 java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class TestSqlFileBasedTransformer extends UtilitiesTestBase {
+  @BeforeAll
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+    UtilitiesTestBase.Helpers.copyToDFS(
+        "delta-streamer-config/sql-file-transformer.sql",
+        UtilitiesTestBase.dfs,
+        UtilitiesTestBase.dfsBasePath + "/sql-file-transformer.sql");
+    UtilitiesTestBase.Helpers.copyToDFS(
+        "delta-streamer-config/sql-file-transformer-invalid.sql",
+        UtilitiesTestBase.dfs,
+        UtilitiesTestBase.dfsBasePath + "/sql-file-transformer-invalid.sql");
+    UtilitiesTestBase.Helpers.copyToDFS(
+        "delta-streamer-config/sql-file-transformer-empty.sql",
+        UtilitiesTestBase.dfs,
+        UtilitiesTestBase.dfsBasePath + "/sql-file-transformer-empty.sql");
+  }
+
+  @AfterAll
+  public static void cleanupClass() {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @Override
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+  }
+
+  @Override
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+  }
+
+  @Test
+  public void testSqlFileBasedTransformer() {
+    TypedProperties props = new TypedProperties();
+    SqlFileBasedTransformer sqlFileTransformer = new SqlFileBasedTransformer();
+    Dataset<Row> inputDatasetRows = getInputDatasetRows();
+    Dataset<Row> emptyDatasetRow = getEmptyDatasetRow();
+
+    // Test if the class throws illegal argument exception when argument not 
present.
+    assertThrows(
+        IllegalArgumentException.class,
+        () -> sqlFileTransformer.apply(jsc, sparkSession, inputDatasetRows, 
props));
+
+    // Test if the class throws hoodie IO exception correctly when given a 
incorrect config.
+    props.setProperty(
+        "hoodie.deltastreamer.transformer.sql.file",
+        UtilitiesTestBase.dfsBasePath + "/non-exist-sql-file.sql");
+    assertThrows(
+        HoodieIOException.class,
+        () -> sqlFileTransformer.apply(jsc, sparkSession, inputDatasetRows, 
props));
+
+    // Test if the SQL file based transformer works as expected for the 
invalid SQL statements.
+    props.setProperty(
+        "hoodie.deltastreamer.transformer.sql.file",
+        UtilitiesTestBase.dfsBasePath + "/sql-file-transformer-invalid.sql");
+    assertThrows(
+        ParseException.class,
+        () -> sqlFileTransformer.apply(jsc, sparkSession, inputDatasetRows, 
props));
+
+    // Test if the SQL file based transformer works as expected for the empty 
SQL statements.
+    props.setProperty(
+        "hoodie.deltastreamer.transformer.sql.file",
+        UtilitiesTestBase.dfsBasePath + "/sql-file-transformer-empty.sql");
+    Dataset<Row> emptyRow = sqlFileTransformer.apply(jsc, sparkSession, 
inputDatasetRows, props);
+    String[] actualRows = 
emptyRow.as(Encoders.STRING()).collectAsList().toArray(new String[0]);
+    String[] expectedRows = emptyDatasetRow.collectAsList().toArray(new 
String[0]);
+    assertArrayEquals(expectedRows, actualRows);
+
+    // Test if the SQL file based transformer works as expected for the 
correct input.
+    props.setProperty(
+        "hoodie.deltastreamer.transformer.sql.file",
+        UtilitiesTestBase.dfsBasePath + "/sql-file-transformer.sql");
+    Dataset<Row> transformedRow =
+        sqlFileTransformer.apply(jsc, sparkSession, inputDatasetRows, props);
+
+    // Called distinct() and sort() to match the transformation in this file:
+    // 
hudi-utilities/src/test/resources/delta-streamer-config/sql-file-transformer.sql
+    expectedRows =
+        inputDatasetRows
+            .distinct()
+            .sort("col1")
+            .as(Encoders.STRING())
+            .collectAsList()
+            .toArray(new String[0]);
+    actualRows = 
transformedRow.as(Encoders.STRING()).collectAsList().toArray(new String[0]);
+    assertArrayEquals(expectedRows, actualRows);
+  }
+
+  private Dataset<Row> getInputDatasetRows() {
+    // Create few rows with duplicate data.
+    List<Row> list = new ArrayList<>();
+    list.add(RowFactory.create("one"));
+    list.add(RowFactory.create("two"));
+    list.add(RowFactory.create("three"));
+    list.add(RowFactory.create("four"));
+    list.add(RowFactory.create("four"));
+    // Create the schema struct.
+    List<org.apache.spark.sql.types.StructField> listOfStructField = new 
ArrayList<>();
+    listOfStructField.add(DataTypes.createStructField("col1", 
DataTypes.StringType, true));
+    StructType structType = DataTypes.createStructType(listOfStructField);
+    SparkSession spark = sparkSession.builder().getOrCreate();

Review comment:
       Can we reuse spark from UtilsTestBase. Bcoz, looks like we are leaking 
the spark session here w/o cleaning it up after the test is complete. 

##########
File path: 
hudi-utilities/src/test/java/org/apache/hudi/utilities/transform/TestSqlFileBasedTransformer.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.utilities.transform;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Encoders;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.parser.ParseException;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+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 java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+public class TestSqlFileBasedTransformer extends UtilitiesTestBase {
+  @BeforeAll
+  public static void initClass() throws Exception {
+    UtilitiesTestBase.initClass();
+    UtilitiesTestBase.Helpers.copyToDFS(
+        "delta-streamer-config/sql-file-transformer.sql",
+        UtilitiesTestBase.dfs,
+        UtilitiesTestBase.dfsBasePath + "/sql-file-transformer.sql");
+    UtilitiesTestBase.Helpers.copyToDFS(
+        "delta-streamer-config/sql-file-transformer-invalid.sql",
+        UtilitiesTestBase.dfs,
+        UtilitiesTestBase.dfsBasePath + "/sql-file-transformer-invalid.sql");
+    UtilitiesTestBase.Helpers.copyToDFS(
+        "delta-streamer-config/sql-file-transformer-empty.sql",
+        UtilitiesTestBase.dfs,
+        UtilitiesTestBase.dfsBasePath + "/sql-file-transformer-empty.sql");
+  }
+
+  @AfterAll
+  public static void cleanupClass() {
+    UtilitiesTestBase.cleanupClass();
+  }
+
+  @Override
+  @BeforeEach
+  public void setup() throws Exception {
+    super.setup();
+  }
+
+  @Override
+  @AfterEach
+  public void teardown() throws Exception {
+    super.teardown();
+  }
+
+  @Test
+  public void testSqlFileBasedTransformer() {

Review comment:
       minor. can we split these into multiple tests instead of one monolith 
tests. From tests runs, it would be easy to narrow down rootcause if there are 
some failing. But big one is hard to detect and we have to check the stacktrace 
and also hard to maintain as time goes. 




-- 
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:
[email protected]


Reply via email to