james-willis commented on code in PR #2637:
URL: https://github.com/apache/sedona/pull/2637#discussion_r2794908164


##########
docs/api/sql/Function.md:
##########
@@ -19,13 +19,19 @@
 
 ## 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`.
 
-!!!Note
-    Jpostal requires at least 2 GB of free disk space to store the data files 
used for address parsing and expanding. The data files are downloaded 
automatically when the function is called for the first time.
+The data directory can be configured via `spark.sedona.libpostal.dataDir`. You 
can point it to a remote filesystem path (HDFS, S3, GCS, ABFS, etc.) such as 
`hdfs:///data/libpostal/` or `s3a://my-bucket/libpostal/`. When using a remote 
path, you must distribute the data to all executors before running queries by 
calling `sc.addFile("hdfs:///data/libpostal/", recursive=True)` (PySpark) or 
`sc.addFile("hdfs:///data/libpostal/", recursive = true)` (Scala). In this 
remote-URI mode, the automatic internet download performed by jpostal is 
disabled, so the remote directory must already contain the libpostal model 
files. For local filesystem paths, jpostal's download-if-needed behavior 
remains enabled.

Review Comment:
   why wouldnt this be automatically distirbuted as part of the implementation 
here?
   



##########
docs/api/sql/Function.md:
##########
@@ -19,13 +19,19 @@
 
 ## 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:
   This should still be in a note, not the intro



##########
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.equalsIgnoreCase("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)

Review Comment:
   in contrast with the docs it seems that this code downloads the data from 
the hdfs itself.



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