You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by si...@apache.org on 2020/05/28 00:18:51 UTC

[hudi] 08/40: [HUDI-656][Performance] Return a dummy Spark relation after writing the DataFrame (#1394)

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

sivabalan pushed a commit to branch release-0.5.3
in repository https://gitbox.apache.org/repos/asf/hudi.git

commit 9a2b3e3c93f670c79adb3d5cba69cdd3f551005a
Author: Udit Mehrotra <um...@illinois.edu>
AuthorDate: Wed Mar 11 20:27:46 2020 -0700

    [HUDI-656][Performance] Return a dummy Spark relation after writing the DataFrame (#1394)
    
    Co-authored-by: Mehrotra <ud...@amazon.com>
---
 .../main/scala/org/apache/hudi/DefaultSource.scala | 20 ++++++++++++-
 .../scala/org/apache/hudi/HudiEmptyRelation.scala  | 35 ++++++++++++++++++++++
 2 files changed, 54 insertions(+), 1 deletion(-)

diff --git a/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala b/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
index 931f23b..fbdd4ea 100644
--- a/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
+++ b/hudi-spark/src/main/scala/org/apache/hudi/DefaultSource.scala
@@ -22,6 +22,7 @@ import org.apache.hudi.exception.HoodieException
 import org.apache.hudi.hadoop.HoodieROTablePathFilter
 import org.apache.log4j.LogManager
 import org.apache.spark.sql.execution.datasources.DataSource
+import org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand
 import org.apache.spark.sql.execution.streaming.Sink
 import org.apache.spark.sql.sources._
 import org.apache.spark.sql.streaming.OutputMode
@@ -82,6 +83,22 @@ class DefaultSource extends RelationProvider
     }
   }
 
+  /**
+    * This DataSource API is used for writing the DataFrame at the destination. For now, we are returning a dummy
+    * relation here because Spark does not really make use of the relation returned, and just returns an empty
+    * dataset at [[SaveIntoDataSourceCommand.run()]]. This saves us the cost of creating and returning a parquet
+    * relation here.
+    *
+    * TODO: Revisit to return a concrete relation here when we support CREATE TABLE AS for Hudi with DataSource API.
+    *       That is the only case where Spark seems to actually need a relation to be returned here
+    *       [[DataSource.writeAndRead()]]
+    *
+    * @param sqlContext Spark SQL Context
+    * @param mode Mode for saving the DataFrame at the destination
+    * @param optParams Parameters passed as part of the DataFrame write operation
+    * @param df Spark DataFrame to be written
+    * @return Spark Relation
+    */
   override def createRelation(sqlContext: SQLContext,
                               mode: SaveMode,
                               optParams: Map[String, String],
@@ -89,7 +106,8 @@ class DefaultSource extends RelationProvider
 
     val parameters = HoodieSparkSqlWriter.parametersWithWriteDefaults(optParams)
     HoodieSparkSqlWriter.write(sqlContext, mode, parameters, df)
-    createRelation(sqlContext, parameters, df.schema)
+
+    new HudiEmptyRelation(sqlContext, df.schema)
   }
 
   override def createSink(sqlContext: SQLContext,
diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HudiEmptyRelation.scala b/hudi-spark/src/main/scala/org/apache/hudi/HudiEmptyRelation.scala
new file mode 100644
index 0000000..8ddbe46
--- /dev/null
+++ b/hudi-spark/src/main/scala/org/apache/hudi/HudiEmptyRelation.scala
@@ -0,0 +1,35 @@
+/*
+ * 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
+
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.sources.BaseRelation
+import org.apache.spark.sql.types.StructType
+
+/**
+  * This is a dummy Spark relation that can be used if needed to return a place holder relation that does not get used.
+  *
+  * @param sqlContext Spark SQL Context
+  * @param userSchema Users data schema
+  */
+class HudiEmptyRelation(val sqlContext: SQLContext,
+                        val userSchema: StructType) extends BaseRelation {
+
+  override def schema: StructType = userSchema
+}