You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "abmo-x (via GitHub)" <gi...@apache.org> on 2023/05/09 19:18:48 UTC

[GitHub] [iceberg] abmo-x commented on a diff in pull request #7363: Spark 3.2: Optimized add_files procedure's listPartitions

abmo-x commented on code in PR #7363:
URL: https://github.com/apache/iceberg/pull/7363#discussion_r1189039431


##########
spark/v3.2/spark/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryPartitionPathsIndex.scala:
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.execution.datasources
+import org.apache.hadoop.fs._
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.types.StructType
+import scala.collection.mutable
+
+
+/**
+ * Implement our own in-memory index which will only list directories to avoid unnecessary file
+ * listings. Should ONLY be used to get partition directory paths. Uses table's schema to only
+ * visit partition dirs using number of partition columns depth recursively. Does NOT return files
+ * within leaf dir.
+ */
+class InMemoryPartitionPathsIndex(
+                         sparkSession: SparkSession,
+                         rootPath :Path,
+                         parameters: Map[String, String],
+                         userSpecifiedSchema: StructType,
+                         fileStatusCache: FileStatusCache = NoopCache,
+                         override val metadataOpsTimeNs: Option[Long] = None)
+  extends PartitioningAwareFileIndex(
+    sparkSession, parameters, Some(userSpecifiedSchema), fileStatusCache) {
+
+  override val rootPaths = Seq(rootPath)
+
+  @volatile private var cachedLeafFiles: mutable.LinkedHashMap[Path, FileStatus] = _
+  @volatile private var cachedLeafDirToChildrenFiles: Map[Path, Array[FileStatus]] = _
+  @volatile private var cachedPartitionSpec: PartitionSpec = _
+
+  refresh0()
+
+  override def partitionSpec(): PartitionSpec = {
+    if (cachedPartitionSpec == null) {
+        cachedPartitionSpec = inferPartitioning()
+    }
+    logTrace(s"Partition spec: $cachedPartitionSpec")
+    cachedPartitionSpec
+  }
+
+  override def allFiles(): Seq[FileStatus] = cachedLeafFiles.values.toSeq
+
+  override protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] = cachedLeafFiles
+
+  override protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] = cachedLeafDirToChildrenFiles
+
+  override def refresh(): Unit = {
+    fileStatusCache.invalidateAll()
+    refresh0()
+  }
+
+  private def refresh0(): Unit = {
+    val files = listLeafDirs(rootPath, 0)
+    cachedLeafFiles =
+      new mutable.LinkedHashMap[Path, FileStatus]() ++= files.map(f => f.getPath -> f)
+    cachedLeafDirToChildrenFiles = files.map( f => (f.getPath, Array(createEmptyChildDataFileStatus(f))))
+      .toMap
+    cachedPartitionSpec = null
+  }
+
+  override def equals(other: Any): Boolean = other match {
+    case hdfs: InMemoryFileIndex => rootPaths.toSet == hdfs.rootPaths.toSet
+    case _ => false
+  }
+
+  override def hashCode(): Int = rootPaths.toSet.hashCode()
+
+  /**
+   * recursively lists only the partition dirs. Uses the number of partition cols

Review Comment:
   @szehon-ho 
   correct, original method may distribute the task to list files.
   
   However, in our implementation as we **don't** list files and _only_ list directories, we don't need parallelism as listing  just dirs will be quicker or not that many compared to listing files in each dir.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org