advancedxy commented on code in PR #9563: URL: https://github.com/apache/iceberg/pull/9563#discussion_r1474613749
########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPlanningUtil.java: ########## @@ -0,0 +1,94 @@ +/* + * 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.iceberg.spark.source; + +import java.util.List; +import java.util.Objects; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionScanTask; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.util.ThreadPools; + +class SparkPlanningUtil { + + public static final String[] NO_LOCATION_PREFERENCE = new String[0]; + + private SparkPlanningUtil() {} + + public static String[][] fetchBlockLocations( + FileIO io, List<? extends ScanTaskGroup<?>> taskGroups) { + String[][] locations = new String[taskGroups.size()][]; + + Tasks.range(taskGroups.size()) + .stopOnFailure() + .executeWith(ThreadPools.getWorkerPool()) + .run(index -> locations[index] = Util.blockLocations(io, taskGroups.get(index))); + + return locations; + } + + public static String[][] assignExecutors( + List<? extends ScanTaskGroup<?>> taskGroups, List<String> executorLocations) { + String[][] locations = new String[taskGroups.size()][]; + + for (int index = 0; index < taskGroups.size(); index++) { + locations[index] = assign(taskGroups.get(index), executorLocations); + } + + return locations; + } + + private static String[] assign(ScanTaskGroup<?> taskGroup, List<String> executorLocations) { + List<String> locations = Lists.newArrayList(); + + for (ScanTask task : taskGroup.tasks()) { + if (task.isFileScanTask()) { + FileScanTask fileTask = task.asFileScanTask(); + if (isPartitioned(fileTask) && !fileTask.deletes().isEmpty()) { + int index = Math.floorMod(hash(fileTask.partition()), executorLocations.size()); + String executorLocation = executorLocations.get(index); + locations.add(executorLocation); + } + } + } + + return locations.toArray(NO_LOCATION_PREFERENCE); + } + + private static boolean isPartitioned(PartitionScanTask task) { + return task.partition() != null && task.partition().size() > 0; Review Comment: how about checks `task.spec().isPartitioned()` instead? Otherwise, `VoidTransform` in V1 table is not handled in the above code. ########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java: ########## @@ -331,4 +331,24 @@ private long driverMaxResultSize() { SparkConf sparkConf = spark.sparkContext().conf(); return sparkConf.getSizeAsBytes(DRIVER_MAX_RESULT_SIZE, DRIVER_MAX_RESULT_SIZE_DEFAULT); } + + public boolean executorCacheLocalityEnabled() { + return executorCacheEnabled() && executorCacheLocalityEnabledInternal(); + } + + private boolean executorCacheEnabled() { + return confParser + .booleanConf() + .sessionConf(SparkSQLProperties.EXECUTOR_CACHE_ENABLED) + .defaultValue(SparkSQLProperties.EXECUTOR_CACHE_ENABLED_DEFAULT) + .parse(); + } + + private boolean executorCacheLocalityEnabledInternal() { Review Comment: I'm a little concerned that this doesn't play well with Spark's `dynamicAllocation` which should be enabled by default for most production systems. Did you test how would this work with dynamic allocation enabled? ########## spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java: ########## @@ -85,6 +86,30 @@ public static void clearTestSparkCatalogCache() { TestSparkCatalog.clearTables(); } + @Test + public void testDeleteWithExecutorCacheLocality() throws NoSuchTableException { + createAndInitPartitionedTable(); + + append(tableName, new Employee(1, "hr"), new Employee(2, "hr")); + append(tableName, new Employee(3, "hr"), new Employee(4, "hr")); + append(tableName, new Employee(1, "hardware"), new Employee(2, "hardware")); + append(tableName, new Employee(3, "hardware"), new Employee(4, "hardware")); + + createBranchIfNeeded(); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.EXECUTOR_CACHE_LOCALITY_ENABLED, "true"), + () -> { + sql("DELETE FROM %s WHERE id = 1", commitTarget()); + sql("DELETE FROM %s WHERE id = 3", commitTarget()); + + assertEquals( Review Comment: Seems it only checks for records equality, but doesn't check the executor cache locality? I think we may check spark RDD's ` getPreferredLocations` instead? ########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPlanningUtil.java: ########## @@ -0,0 +1,94 @@ +/* + * 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.iceberg.spark.source; + +import java.util.List; +import java.util.Objects; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionScanTask; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.util.ThreadPools; + +class SparkPlanningUtil { + + public static final String[] NO_LOCATION_PREFERENCE = new String[0]; + + private SparkPlanningUtil() {} + + public static String[][] fetchBlockLocations( + FileIO io, List<? extends ScanTaskGroup<?>> taskGroups) { + String[][] locations = new String[taskGroups.size()][]; + + Tasks.range(taskGroups.size()) + .stopOnFailure() + .executeWith(ThreadPools.getWorkerPool()) + .run(index -> locations[index] = Util.blockLocations(io, taskGroups.get(index))); + + return locations; + } + + public static String[][] assignExecutors( + List<? extends ScanTaskGroup<?>> taskGroups, List<String> executorLocations) { + String[][] locations = new String[taskGroups.size()][]; + + for (int index = 0; index < taskGroups.size(); index++) { + locations[index] = assign(taskGroups.get(index), executorLocations); + } + + return locations; + } + + private static String[] assign(ScanTaskGroup<?> taskGroup, List<String> executorLocations) { + List<String> locations = Lists.newArrayList(); + + for (ScanTask task : taskGroup.tasks()) { + if (task.isFileScanTask()) { + FileScanTask fileTask = task.asFileScanTask(); + if (isPartitioned(fileTask) && !fileTask.deletes().isEmpty()) { + int index = Math.floorMod(hash(fileTask.partition()), executorLocations.size()); + String executorLocation = executorLocations.get(index); + locations.add(executorLocation); + } + } + } + + return locations.toArray(NO_LOCATION_PREFERENCE); + } + + private static boolean isPartitioned(PartitionScanTask task) { + return task.partition() != null && task.partition().size() > 0; + } + + private static int hash(StructLike struct) { Review Comment: is it possible to reuse `org.apache.iceberg.types.JavaHash`(es) here? -- 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-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org