You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "jaceklaskowski (via GitHub)" <gi...@apache.org> on 2023/04/20 13:01:32 UTC

[GitHub] [spark] jaceklaskowski commented on a diff in pull request #40821: [SPARK-43152][spark-structured-streaming] Parametrisable output metadata path (_spark_metadata)

jaceklaskowski commented on code in PR #40821:
URL: https://github.com/apache/spark/pull/40821#discussion_r1172542799


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -1864,6 +1864,13 @@ object SQLConf {
       .checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2")
       .createWithDefault(2)
 
+  val CUSTOM_OUTPUT_METADATA_PATH = buildConf("spark.sql.streaming.metadata.output.path")
+      .doc("The place where _spark_metadata folder will be created " +

Review Comment:
   s/place/directory
   s/will be created and used for/for
   s/of streaming job/of a streaming query
   
   



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -1864,6 +1864,13 @@ object SQLConf {
       .checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2")
       .createWithDefault(2)
 
+  val CUSTOM_OUTPUT_METADATA_PATH = buildConf("spark.sql.streaming.metadata.output.path")

Review Comment:
   nit: Replace `CUSTOM` with `STREAMING` or remove altogether (perhaps also rearranging the parts to match the name of the conf)?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala:
##########
@@ -68,8 +68,15 @@ object FileStreamSink extends Logging {
     }
   }
 
-  def getMetadataLogPath(fs: FileSystem, path: Path, sqlConf: SQLConf): Path = {
-    val metadataDir = new Path(path, FileStreamSink.metadataDir)
+  def getMetadataLogPath(
+      hadoopConfiguration: Configuration,
+      path: Path,
+      sqlConf: SQLConf): Path = {
+    val metadataDir = sqlConf.streamingMetadataOutputPath match {
+      case Some(customPath) => new Path(customPath, FileStreamSink.metadataDir)
+      case None => new Path(path, FileStreamSink.metadataDir)
+    }

Review Comment:
   ```
   val metadataPath = sqlConf.streamingMetadataOutputPath.getOrElse(path)
   val metadataDir = new Path(metadataPath, FileStreamSink.metadataDir)
   ```



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala:
##########
@@ -126,14 +133,15 @@ class FileStreamSink(
     path: String,
     fileFormat: FileFormat,
     partitionColumnNames: Seq[String],
-    options: Map[String, String]) extends Sink with Logging {
+    options: Map[String, String])
+    extends Sink

Review Comment:
   Shouldn't it be left-indented (if at all introduced in the PR)?



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala:
##########
@@ -649,6 +649,28 @@ abstract class FileStreamSinkSuite extends StreamTest {
       }
     }
   }
+  test("SPARK-43152: Support parametrisable output metadata path (_spark_metadata)") {

Review Comment:
   s/parametrisable/user-defined



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -4385,6 +4392,8 @@ class SQLConf extends Serializable with Logging {
   def stateStoreSkipNullsForStreamStreamJoins: Boolean =
     getConf(STATE_STORE_SKIP_NULLS_FOR_STREAM_STREAM_JOINS)
 
+  def streamingMetadataOutputPath: Option[String] = getConf(SQLConf.CUSTOM_OUTPUT_METADATA_PATH)

Review Comment:
   Do we need this `streaming` prefix (since it wasn't needed for the constant itself)? Be consistent 🙏 



-- 
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: reviews-unsubscribe@spark.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org