Copilot commented on code in PR #2637:
URL: https://github.com/apache/sedona/pull/2637#discussion_r2791871909


##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/LibPostalDataLoader.scala:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.spark.sql.sedona_sql.expressions
+
+import org.apache.spark.SparkFiles
+import org.slf4j.LoggerFactory
+
+import java.io.File
+import java.net.URI
+
+/**
+ * Resolves libpostal data directory paths. When the configured data directory 
points to a remote
+ * filesystem (HDFS, S3, GCS, ABFS, etc.), the data is expected to have been 
distributed to
+ * executors via `SparkContext.addFile()` and is resolved through 
`SparkFiles.get()`.
+ */
+object LibPostalDataLoader {
+
+  private val logger = LoggerFactory.getLogger(getClass)
+
+  /**
+   * Resolve the data directory to a local filesystem path. If the configured 
path already points
+   * to the local filesystem, it is returned as-is. If it points to a remote 
filesystem, the data
+   * is looked up via Spark's `SparkFiles` mechanism (the user must have called
+   * `sc.addFile(remotePath, recursive = true)` before running queries).
+   *
+   * @param configuredDir
+   *   the data directory path from Sedona configuration (may be local or 
remote)
+   * @return
+   *   a local filesystem path suitable for jpostal
+   */
+  def resolveDataDir(configuredDir: String): String = {
+    if (isRemotePath(configuredDir)) {
+      resolveFromSparkFiles(configuredDir)
+    } else {
+      normalizeLocalPath(configuredDir)
+    }
+  }
+
+  /**
+   * Normalize a local path. Converts `file:` URIs (e.g. 
`file:///tmp/libpostal`) to plain
+   * filesystem paths (`/tmp/libpostal`) so that jpostal receives a path it 
can use directly.
+   * Non-URI paths are returned unchanged.
+   */
+  private[expressions] def normalizeLocalPath(path: String): String = {
+    try {
+      val uri = new URI(path)
+      if (uri.getScheme != null && uri.getScheme.equalsIgnoreCase("file")) {
+        new File(uri).getAbsolutePath
+      } else {
+        path
+      }
+    } catch {
+      case _: Exception => path
+    }
+  }
+
+  /**
+   * Determine whether a path string refers to a remote (non-local) filesystem.
+   */
+  def isRemotePath(path: String): Boolean = {
+    try {
+      val uri = new URI(path)
+      val scheme = uri.getScheme
+      scheme != null && scheme != "file" && scheme.length > 1
+    } catch {
+      case _: Exception => false
+    }
+  }

Review Comment:
   `isRemotePath` compares `scheme != "file"` case-sensitively, so inputs like 
`FILE:///tmp/libpostal` would be incorrectly treated as remote. Use a 
case-insensitive check (e.g., `!scheme.equalsIgnoreCase("file")`) to match 
`normalizeLocalPath`’s behavior.



##########
docs/api/sql/Function.md:
##########
@@ -19,13 +19,9 @@
 
 ## ExpandAddress
 
-Introduction: Returns an array of expanded forms of the input address string. 
This is backed by the [libpostal](https://github.com/openvenues/libpostal) 
library's address expanding functionality.
+Introduction: Returns an array of expanded forms of the input address string. 
This is backed by the [libpostal](https://github.com/openvenues/libpostal) 
library's address expanding functionality. Jpostal requires at least 2 GB of 
free disk space to store the data files used for address parsing and expanding. 
By default, the data files are downloaded automatically to a temporary 
directory (`<java.io.tmpdir>/libpostal/`, e.g. `/tmp/libpostal/` on 
Linux/macOS) when the function is called for the first time. The version of 
jpostal installed with this package only supports Linux and MacOS. If you are 
using Windows, you will need to install libjpostal and libpostal manually and 
ensure that they are available in your `java.library.path`.

Review Comment:
   Use the standard capitalization `macOS` instead of `MacOS`.



##########
spark/common/src/test/scala/org/apache/sedona/sql/LibPostalDataLoaderTest.scala:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.sedona.sql
+
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.spark.sql.sedona_sql.expressions.LibPostalDataLoader
+import org.scalatest.matchers.should.Matchers
+
+import java.io.File
+import java.nio.file.Files
+
+class LibPostalDataLoaderTest extends TestBaseScala with Matchers {
+
+  describe("LibPostalDataLoader") {
+
+    describe("isRemotePath") {
+      it("should return false for local paths") {
+        LibPostalDataLoader.isRemotePath("/tmp/libpostal/") shouldBe false
+      }
+
+      it("should return false for relative paths") {
+        LibPostalDataLoader.isRemotePath("data/libpostal/") shouldBe false
+      }
+
+      it("should return false for file:// URIs") {
+        LibPostalDataLoader.isRemotePath("file:///tmp/libpostal/") shouldBe 
false
+      }
+
+      it("should return true for hdfs:// URIs") {
+        LibPostalDataLoader.isRemotePath("hdfs:///data/libpostal/") shouldBe 
true
+      }
+
+      it("should return true for hdfs:// URIs with host") {
+        
LibPostalDataLoader.isRemotePath("hdfs://namenode:9000/data/libpostal/") 
shouldBe true
+      }
+
+      it("should return true for s3a:// URIs") {
+        LibPostalDataLoader.isRemotePath("s3a://my-bucket/libpostal/") 
shouldBe true
+      }
+
+      it("should return true for s3:// URIs") {
+        LibPostalDataLoader.isRemotePath("s3://my-bucket/libpostal/") shouldBe 
true
+      }
+
+      it("should return true for gs:// URIs") {
+        LibPostalDataLoader.isRemotePath("gs://my-bucket/libpostal/") shouldBe 
true
+      }
+
+      it("should return true for abfs:// URIs") {
+        LibPostalDataLoader.isRemotePath(
+          "abfs://[email protected]/libpostal/") shouldBe 
true
+      }
+
+      it("should return true for wasb:// URIs") {
+        LibPostalDataLoader.isRemotePath(
+          "wasb://[email protected]/libpostal/") 
shouldBe true
+      }
+
+      it("should return false for empty string") {
+        LibPostalDataLoader.isRemotePath("") shouldBe false
+      }
+
+      it("should return false for Windows-like paths") {
+        // Single-letter scheme like C: should not be treated as remote
+        LibPostalDataLoader.isRemotePath("C:\\libpostal\\data\\") shouldBe 
false
+      }
+    }
+
+    describe("resolveDataDir") {
+      it("should return local path unchanged") {
+        val tempDir = Files.createTempDirectory("sedona-libpostal-test").toFile
+        try {
+          val result = 
LibPostalDataLoader.resolveDataDir(tempDir.getAbsolutePath)
+          result shouldBe tempDir.getAbsolutePath
+        } finally {
+          tempDir.delete()
+        }
+      }
+
+      it("should normalize file: URI to plain local path") {
+        val tempDir = Files.createTempDirectory("sedona-libpostal-test").toFile
+        try {
+          val fileUri = tempDir.toURI.toString
+          val result = LibPostalDataLoader.resolveDataDir(fileUri)
+          result should not startWith "file:"
+          result shouldBe tempDir.getAbsolutePath
+        } finally {
+          tempDir.delete()
+        }
+      }
+
+      it("should normalize file: URI without trailing slash") {
+        val result = 
LibPostalDataLoader.resolveDataDir("file:///tmp/libpostal")
+        result should not startWith "file:"
+        result shouldBe "/tmp/libpostal"
+      }
+
+      it("should resolve remote path via SparkFiles when data was added") {
+        // Create a local temp directory simulating libpostal data
+        val tempDir = 
Files.createTempDirectory("sedona-libpostal-sparkfiles-test").toFile
+        val subDir = new File(tempDir, "address_parser")
+        subDir.mkdirs()
+        new File(subDir, "model.dat").createNewFile()
+
+        try {
+          // Use sc.addFile to distribute the directory
+          sc.addFile(tempDir.getAbsolutePath, true)
+
+          // Construct a remote-style URI using the basename
+          val basename = tempDir.getName
+          val remoteUri = s"hdfs:///data/$basename/"
+
+          val result = LibPostalDataLoader.resolveDataDir(remoteUri)
+
+          // The resolved path should be a local path from SparkFiles
+          result should not startWith "hdfs://"
+          new File(result).exists() shouldBe true
+          new File(result).isDirectory shouldBe true
+          result should endWith(File.separator)
+        } finally {
+          new File(tempDir, "address_parser/model.dat").delete()
+          subDir.delete()
+          tempDir.delete()
+        }
+      }
+
+      it("should throw IllegalStateException when remote data not found in 
SparkFiles") {
+        val remoteUri = "hdfs:///data/nonexistent-libpostal-data/"
+
+        val exception = intercept[IllegalStateException] {
+          LibPostalDataLoader.resolveDataDir(remoteUri)
+        }
+        exception.getMessage should include("not found via SparkFiles")
+        exception.getMessage should include("sc.addFile")
+      }
+
+      it("should resolve data uploaded to HDFS via sc.addFile end-to-end") {
+        val (hdfsCluster, hdfsUri) = creatMiniHdfs()

Review Comment:
   The helper name `creatMiniHdfs()` looks like a typo; if the actual helper is 
named `createMiniHdfs()` (common naming), this will fail to compile. Consider 
renaming the call (or the helper) to the correct spelling for consistency.
   ```suggestion
           val (hdfsCluster, hdfsUri) = createMiniHdfs()
   ```



##########
docs/api/sql/Function.md:
##########
@@ -45,13 +41,9 @@ Output:
 
 ## ParseAddress
 
-Introduction: Returns an array of the components (e.g. street, postal code) of 
the input address string. This is backed by the 
[libpostal](https://github.com/openvenues/libpostal) library's address parsing 
functionality.
+Introduction: Returns an array of the components (e.g. street, postal code) of 
the input address string. This is backed by the 
[libpostal](https://github.com/openvenues/libpostal) library's address parsing 
functionality. Jpostal requires at least 2 GB of free disk space to store the 
data files used for address parsing and expanding. By default, the data files 
are downloaded automatically to a temporary directory 
(`<java.io.tmpdir>/libpostal/`, e.g. `/tmp/libpostal/` on Linux/macOS) when the 
library is initialized. The version of jpostal installed with this package only 
supports Linux and MacOS. If you are using Windows, you will need to install 
libjpostal and libpostal manually and ensure that they are available in your 
`java.library.path`.

Review Comment:
   Use the standard capitalization `macOS` instead of `MacOS`.



##########
spark/common/src/test/scala/org/apache/sedona/sql/LibPostalDataLoaderTest.scala:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.sedona.sql
+
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.spark.sql.sedona_sql.expressions.LibPostalDataLoader
+import org.scalatest.matchers.should.Matchers
+
+import java.io.File
+import java.nio.file.Files
+
+class LibPostalDataLoaderTest extends TestBaseScala with Matchers {
+
+  describe("LibPostalDataLoader") {
+
+    describe("isRemotePath") {
+      it("should return false for local paths") {
+        LibPostalDataLoader.isRemotePath("/tmp/libpostal/") shouldBe false
+      }
+
+      it("should return false for relative paths") {
+        LibPostalDataLoader.isRemotePath("data/libpostal/") shouldBe false
+      }
+
+      it("should return false for file:// URIs") {
+        LibPostalDataLoader.isRemotePath("file:///tmp/libpostal/") shouldBe 
false
+      }
+
+      it("should return true for hdfs:// URIs") {
+        LibPostalDataLoader.isRemotePath("hdfs:///data/libpostal/") shouldBe 
true
+      }
+
+      it("should return true for hdfs:// URIs with host") {
+        
LibPostalDataLoader.isRemotePath("hdfs://namenode:9000/data/libpostal/") 
shouldBe true
+      }
+
+      it("should return true for s3a:// URIs") {
+        LibPostalDataLoader.isRemotePath("s3a://my-bucket/libpostal/") 
shouldBe true
+      }
+
+      it("should return true for s3:// URIs") {
+        LibPostalDataLoader.isRemotePath("s3://my-bucket/libpostal/") shouldBe 
true
+      }
+
+      it("should return true for gs:// URIs") {
+        LibPostalDataLoader.isRemotePath("gs://my-bucket/libpostal/") shouldBe 
true
+      }
+
+      it("should return true for abfs:// URIs") {
+        LibPostalDataLoader.isRemotePath(
+          "abfs://[email protected]/libpostal/") shouldBe 
true
+      }
+
+      it("should return true for wasb:// URIs") {
+        LibPostalDataLoader.isRemotePath(
+          "wasb://[email protected]/libpostal/") 
shouldBe true
+      }
+
+      it("should return false for empty string") {
+        LibPostalDataLoader.isRemotePath("") shouldBe false
+      }
+
+      it("should return false for Windows-like paths") {
+        // Single-letter scheme like C: should not be treated as remote
+        LibPostalDataLoader.isRemotePath("C:\\libpostal\\data\\") shouldBe 
false
+      }
+    }
+
+    describe("resolveDataDir") {
+      it("should return local path unchanged") {
+        val tempDir = Files.createTempDirectory("sedona-libpostal-test").toFile
+        try {
+          val result = 
LibPostalDataLoader.resolveDataDir(tempDir.getAbsolutePath)
+          result shouldBe tempDir.getAbsolutePath
+        } finally {
+          tempDir.delete()
+        }
+      }
+
+      it("should normalize file: URI to plain local path") {
+        val tempDir = Files.createTempDirectory("sedona-libpostal-test").toFile
+        try {
+          val fileUri = tempDir.toURI.toString
+          val result = LibPostalDataLoader.resolveDataDir(fileUri)
+          result should not startWith "file:"
+          result shouldBe tempDir.getAbsolutePath
+        } finally {
+          tempDir.delete()
+        }
+      }
+
+      it("should normalize file: URI without trailing slash") {
+        val result = 
LibPostalDataLoader.resolveDataDir("file:///tmp/libpostal")
+        result should not startWith "file:"
+        result shouldBe "/tmp/libpostal"

Review Comment:
   This assertion hard-codes a POSIX path (`/tmp/libpostal`), which makes the 
test platform-dependent (path separators / absolute path format differ on 
Windows). Prefer asserting against a computed expected value derived from the 
same URI (e.g., using `new File(new URI(...)).getAbsolutePath`) rather than a 
literal string.
   ```suggestion
           val tempDir = 
Files.createTempDirectory("sedona-libpostal-notrailing-test").toFile
           try {
             val fileUriWithoutSlash = "file:" + tempDir.getAbsolutePath
             val result = 
LibPostalDataLoader.resolveDataDir(fileUriWithoutSlash)
             result should not startWith "file:"
             result shouldBe tempDir.getAbsolutePath
           } finally {
             tempDir.delete()
           }
   ```



##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/LibPostalDataLoader.scala:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.spark.sql.sedona_sql.expressions
+
+import org.apache.spark.SparkFiles
+import org.slf4j.LoggerFactory
+
+import java.io.File
+import java.net.URI
+
+/**
+ * Resolves libpostal data directory paths. When the configured data directory 
points to a remote
+ * filesystem (HDFS, S3, GCS, ABFS, etc.), the data is expected to have been 
distributed to
+ * executors via `SparkContext.addFile()` and is resolved through 
`SparkFiles.get()`.
+ */
+object LibPostalDataLoader {
+
+  private val logger = LoggerFactory.getLogger(getClass)
+
+  /**
+   * Resolve the data directory to a local filesystem path. If the configured 
path already points
+   * to the local filesystem, it is returned as-is. If it points to a remote 
filesystem, the data
+   * is looked up via Spark's `SparkFiles` mechanism (the user must have called
+   * `sc.addFile(remotePath, recursive = true)` before running queries).
+   *
+   * @param configuredDir
+   *   the data directory path from Sedona configuration (may be local or 
remote)
+   * @return
+   *   a local filesystem path suitable for jpostal
+   */
+  def resolveDataDir(configuredDir: String): String = {
+    if (isRemotePath(configuredDir)) {
+      resolveFromSparkFiles(configuredDir)
+    } else {
+      normalizeLocalPath(configuredDir)
+    }
+  }
+
+  /**
+   * Normalize a local path. Converts `file:` URIs (e.g. 
`file:///tmp/libpostal`) to plain
+   * filesystem paths (`/tmp/libpostal`) so that jpostal receives a path it 
can use directly.
+   * Non-URI paths are returned unchanged.
+   */
+  private[expressions] def normalizeLocalPath(path: String): String = {
+    try {
+      val uri = new URI(path)
+      if (uri.getScheme != null && uri.getScheme.equalsIgnoreCase("file")) {
+        new File(uri).getAbsolutePath
+      } else {
+        path
+      }
+    } catch {
+      case _: Exception => path
+    }
+  }
+
+  /**
+   * Determine whether a path string refers to a remote (non-local) filesystem.
+   */
+  def isRemotePath(path: String): Boolean = {
+    try {
+      val uri = new URI(path)
+      val scheme = uri.getScheme
+      scheme != null && scheme != "file" && scheme.length > 1
+    } catch {
+      case _: Exception => false
+    }
+  }
+
+  /**
+   * Resolve a remote data directory via Spark's file distribution mechanism. 
Extracts the
+   * basename (last path component) from the remote URI and looks it up 
through `SparkFiles.get`.
+   * The user must have previously called `sc.addFile(remotePath, recursive = 
true)`.
+   *
+   * @throws IllegalStateException
+   *   if the data directory was not found via SparkFiles
+   */
+  private def resolveFromSparkFiles(remotePath: String): String = {
+    val basename = extractBasename(remotePath)
+
+    try {
+      val localPath = SparkFiles.get(basename)
+      val localFile = new File(localPath)
+
+      if (localFile.exists() && localFile.isDirectory) {
+        logger.info(
+          "Resolved libpostal data from SparkFiles: {} -> {}",
+          remotePath: Any,
+          localPath: Any)
+        ensureTrailingSlash(localPath)
+      } else {
+        throw new IllegalStateException(
+          s"Libpostal data directory '$basename' was not found via SparkFiles. 
" +
+            "Please call sc.addFile(\"" + remotePath + "\", true) before 
running libpostal queries.")
+      }
+    } catch {
+      case e: IllegalStateException => throw e
+      case e: Exception =>
+        throw new IllegalStateException(
+          s"Failed to resolve libpostal data from SparkFiles for 
'$remotePath'. " +
+            "Please call sc.addFile(\"" + remotePath + "\", true) before 
running libpostal queries.",

Review Comment:
   The error message is helpful but has minor consistency issues: it uses 
`Libpostal` (capitalization differs from `libpostal` elsewhere), and it’s not 
explicit that the `true` argument is the *recursive* flag. Consider 
standardizing the product name capitalization and tweaking the message to 
explicitly call out “recursive = true” to reduce confusion.
   ```suggestion
             s"libpostal data directory '$basename' was not found via 
SparkFiles. " +
               "Please call sc.addFile(\"" + remotePath + "\", recursive = 
true) before running libpostal queries.")
         }
       } catch {
         case e: IllegalStateException => throw e
         case e: Exception =>
           throw new IllegalStateException(
             s"Failed to resolve libpostal data from SparkFiles for 
'$remotePath'. " +
               "Please call sc.addFile(\"" + remotePath + "\", recursive = 
true) before running libpostal queries.",
   ```



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

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to