You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/12/06 23:42:34 UTC

[GitHub] [iceberg] aokolnychyi opened a new pull request, #6371: Spark 3.3: Support storage-partitioned joins

aokolnychyi opened a new pull request, #6371:
URL: https://github.com/apache/iceberg/pull/6371

   This PR adds support for storage-partitioned joins in Spark 3.3.


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055944636


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java:
##########
@@ -473,6 +490,10 @@ private static String parentName(String[] fieldNames) {
     return null;
   }
 
+  public static String describe(List<org.apache.iceberg.expressions.Expression> exprs) {
+    return exprs.stream().map(Spark3Util::describe).collect(Collectors.joining(", "));

Review Comment:
   Actually, this is used by `toString`, so it looks like the right behavior is to not sanitize because this is put into Spark plans where the full filter should be shown. We may want to introduce a sanitized describe later though, for logging purposes.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053559291


##########
core/src/main/java/org/apache/iceberg/Partitioning.java:
##########
@@ -215,11 +225,15 @@ public Void alwaysNull(int fieldId, String sourceName, int sourceId) {
    * that have the same field ID but use a void transform under the hood. Such fields cannot be part
    * of the grouping key as void transforms always return null.
    *
+   * <p>If the provided schema is not null, this method will only take into account partition fields
+   * on top of columns present in the schema. Otherwise, all partition fields will be considered.
+   *
+   * @param schema a schema specifying a set of source columns to consider (null to consider all)
    * @param specs one or many specs
    * @return the constructed grouping key type
    */
-  public static StructType groupingKeyType(Collection<PartitionSpec> specs) {
-    return buildPartitionProjectionType("grouping key", specs, commonActiveFieldIds(specs));
+  public static StructType groupingKeyType(Schema schema, Collection<PartitionSpec> specs) {
+    return buildPartitionProjectionType("grouping key", specs, commonActiveFieldIds(schema, specs));

Review Comment:
   It is determined based on scan tasks that match our filter in `SparkPartitioningAwareScan`.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1052490804


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    Preconditions.checkState(taskGroups() != null, "Task groups must be planned");
+
+    if (groupingKeyType().fields().isEmpty()) {
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());

Review Comment:
   We could also bubble up the grouping key type in the scan description.
   It will be shown in the UI too. Let me do that.



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042402581


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(

Review Comment:
   How many of these properties are required because of current Spark limitations?



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042453631


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =

Review Comment:
   Name is fine to me, Ideally this isn't something that get's actually configured by the end user



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


[GitHub] [iceberg] sunchao commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042549493


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    Preconditions.checkState(taskGroups() != null, "Task groups must be planned");
+
+    if (groupingKeyType().fields().isEmpty()) {
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());

Review Comment:
   I don't think we have this info now, but it's a good idea to add it. Let me take a note to implement that on Spark side.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042644217


##########
core/src/main/java/org/apache/iceberg/Partitioning.java:
##########
@@ -215,11 +225,12 @@ public Void alwaysNull(int fieldId, String sourceName, int sourceId) {
    * that have the same field ID but use a void transform under the hood. Such fields cannot be part
    * of the grouping key as void transforms always return null.
    *
+   * @param schema a schema

Review Comment:
   Will add.



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042444111


##########
core/src/main/java/org/apache/iceberg/Partitioning.java:
##########
@@ -215,11 +225,12 @@ public Void alwaysNull(int fieldId, String sourceName, int sourceId) {
    * that have the same field ID but use a void transform under the hood. Such fields cannot be part
    * of the grouping key as void transforms always return null.
    *
+   * @param schema a schema

Review Comment:
   I think this needs a little more description as well as information about what "null" means in this context. The Schema here is the projected schema to pull transforms from?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042644545


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   At the moment, though, we can only support Iceberg to Iceberg joins.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1052657044


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    Preconditions.checkState(taskGroups() != null, "Task groups must be planned");
+
+    if (groupingKeyType().fields().isEmpty()) {
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());

Review Comment:
   I added the grouping key to our scan description. It is now part of the UI. Resolving this thread.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1052659059


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(

Review Comment:
   Done. Resolving.



##########
core/src/main/java/org/apache/iceberg/Partitioning.java:
##########
@@ -215,11 +225,12 @@ public Void alwaysNull(int fieldId, String sourceName, int sourceId) {
    * that have the same field ID but use a void transform under the hood. Such fields cannot be part
    * of the grouping key as void transforms always return null.
    *
+   * @param schema a schema

Review Comment:
   Done. Resolving.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053563618


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java:
##########
@@ -143,4 +148,32 @@ public synchronized void testDeleteWithConcurrentTableRefresh() throws Exception
     executorService.shutdown();
     Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES));
   }
+
+  @Test
+  public void testRuntimeFilteringWithReportedPartitioning() throws NoSuchTableException {
+    createAndInitPartitionedTable();
+
+    append(new Employee(1, "hr"), new Employee(3, "hr"));
+    append(new Employee(1, "hardware"), new Employee(2, "hardware"));
+
+    Map<String, String> sqlConf =
+        ImmutableMap.of(
+            SQLConf.V2_BUCKETING_ENABLED().key(),
+            "true",
+            SparkSQLProperties.PRESERVE_DATA_GROUPING,
+            "true");
+
+    withSQLConf(sqlConf, () -> sql("DELETE FROM %s WHERE id = 2", tableName));
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots()));
+
+    Snapshot currentSnapshot = table.currentSnapshot();
+    validateCopyOnWrite(currentSnapshot, "1", "1", "1");

Review Comment:
   This is what we did historically to simplify the validation as we compare string values. I'll check.



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


[GitHub] [iceberg] aokolnychyi commented on pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#issuecomment-1364570187

   Thanks for reviewing, @RussellSpitzer @sunchao @zinking @rdblue!
   


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1056023188


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java:
##########
@@ -473,6 +490,10 @@ private static String parentName(String[] fieldNames) {
     return null;
   }
 
+  public static String describe(List<org.apache.iceberg.expressions.Expression> exprs) {
+    return exprs.stream().map(Spark3Util::describe).collect(Collectors.joining(", "));

Review Comment:
   That's exactly my thought process too. I wanted to sanitize first but then realized it is used in the plan and kept the existing behavior.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055944028


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java:
##########
@@ -473,6 +490,10 @@ private static String parentName(String[] fieldNames) {
     return null;
   }
 
+  public static String describe(List<org.apache.iceberg.expressions.Expression> exprs) {
+    return exprs.stream().map(Spark3Util::describe).collect(Collectors.joining(", "));

Review Comment:
   Not a blocker, but should we sanitize the expressions using `ExpressionUtil.sanitize` here? There is a risk of logging IDs if we log this.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041608874


##########
core/src/main/java/org/apache/iceberg/Partitioning.java:
##########
@@ -215,11 +225,12 @@ public Void alwaysNull(int fieldId, String sourceName, int sourceId) {
    * that have the same field ID but use a void transform under the hood. Such fields cannot be part
    * of the grouping key as void transforms always return null.
    *
+   * @param schema a schema
    * @param specs one or many specs
    * @return the constructed grouping key type
    */
-  public static StructType groupingKeyType(Collection<PartitionSpec> specs) {
-    return buildPartitionProjectionType("grouping key", specs, commonActiveFieldIds(specs));
+  public static StructType groupingKeyType(Schema schema, Collection<PartitionSpec> specs) {

Review Comment:
   We need to take into account the schema as we may project only particular columns.



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


[GitHub] [iceberg] sunchao commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042680221


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   For me, the question is how to repartition the UnknownDF.
   
   On the Iceberg side, we have a set of N partitions with distinct keys. Now, we need to map the rows UnknownDF into these N partitions. However, at runtime, there could be some rows from UnknownDF that do not belong to any of these partitions.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041611507


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =

Review Comment:
   There were multiple names I considered and `spark.sql.iceberg.split.preserve-partition-boundaries` was one of them. I discarded it because we are not really preserving partition boundaries if there are multiple specs. Also, Spark will push down join keys in the future so we won't really respect partition boundaries as such.
   
   That being said, I'll think more tomorrow. Ideas are always welcome.
   
   



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


[GitHub] [iceberg] sunchao commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042555728


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   In theory it's possible. Spark will check if both sides of the SPJ have compatible partition transforms, via checking whether the V2 function identifiers are the same. So if the non-Iceberg source reports the same functions to Spark and use them in partition transforms, it could work (although I'm not sure whether this is a common use case).



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


[GitHub] [iceberg] sunchao commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042667583


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   What is the partitioning used by the other side?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053557589


##########
api/src/main/java/org/apache/iceberg/types/Types.java:
##########
@@ -554,6 +554,10 @@ public List<NestedField> fields() {
       return lazyFieldList();
     }
 
+    public boolean containsField(int id) {

Review Comment:
   I am using it in `SparkPartitioningAwareScan`.
   
   ```
   if (groupingKeyType().containsField(fieldId) && !seenFieldIds.contains(fieldId)) {
     groupingKeyFields.add(field);
   }
   ```
   
   I could use `groupingKeyType().field(fieldId) != null` but that seems harder to read. Especially, if the condition has multiple branches.
   



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053521508


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java:
##########
@@ -143,4 +148,32 @@ public synchronized void testDeleteWithConcurrentTableRefresh() throws Exception
     executorService.shutdown();
     Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES));
   }
+
+  @Test
+  public void testRuntimeFilteringWithReportedPartitioning() throws NoSuchTableException {
+    createAndInitPartitionedTable();
+
+    append(new Employee(1, "hr"), new Employee(3, "hr"));
+    append(new Employee(1, "hardware"), new Employee(2, "hardware"));
+
+    Map<String, String> sqlConf =
+        ImmutableMap.of(
+            SQLConf.V2_BUCKETING_ENABLED().key(),
+            "true",
+            SparkSQLProperties.PRESERVE_DATA_GROUPING,
+            "true");
+
+    withSQLConf(sqlConf, () -> sql("DELETE FROM %s WHERE id = 2", tableName));
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots()));
+
+    Snapshot currentSnapshot = table.currentSnapshot();
+    validateCopyOnWrite(currentSnapshot, "1", "1", "1");

Review Comment:
   Nit: it would be nice to not pass the counts as strings...



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053515378


##########
core/src/main/java/org/apache/iceberg/Partitioning.java:
##########
@@ -200,6 +200,16 @@ public Void alwaysNull(int fieldId, String sourceName, int sourceId) {
   /**
    * Builds a grouping key type considering all provided specs.
    *
+   * @param specs one or many specs
+   * @return the constructed grouping key type
+   */
+  public static StructType groupingKeyType(Collection<PartitionSpec> specs) {

Review Comment:
   Do we consider this deprecated? When would we want to use it without passing Schema?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1052674043


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {

Review Comment:
   One debatable point if `groupingKeyType` should be part of `equals` and `hashCode`.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055945851


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java:
##########
@@ -356,4 +357,23 @@ private <T> GenericArrayData fillArray(
 
     return new GenericArrayData(array);
   }
+
+  @Override
+  public boolean equals(Object other) {

Review Comment:
   What is the contract for equals for `InternalRow`? This probably works, but does it indicate a bug in Spark where Spark assumes that `InternalRow` implementations consider one another equal?



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042394528


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    Preconditions.checkState(taskGroups() != null, "Task groups must be planned");
+
+    if (groupingKeyType().fields().isEmpty()) {
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());

Review Comment:
   Is this present in the SparkUI anywhere? Feel like we should have some way of knowing whether or not the partitioning was successfully applied



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041613180


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    Preconditions.checkState(taskGroups() != null, "Task groups must be planned");
+
+    if (groupingKeyType().fields().isEmpty()) {
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType =
+            org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    Set<Integer> groupingKeyFieldIds =
+        groupingKeyType().fields().stream()
+            .map(Types.NestedField::fieldId)
+            .collect(Collectors.toSet());
+
+    List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+    for (PartitionSpec spec : specs()) {
+      for (PartitionField field : spec.fields()) {
+        if (groupingKeyFieldIds.contains(field.fieldId())) {
+          groupingKeyFields.add(field);
+          groupingKeyFieldIds.remove(field.fieldId());
+        }
+      }
+    }
+
+    return Spark3Util.toTransforms(table().schema(), groupingKeyFields);
+  }
+
+  protected Set<PartitionSpec> specs() {
+    if (specs == null) {
+      Set<PartitionSpec> taskSpecs = Sets.newHashSet();
+      for (T task : tasks()) {
+        taskSpecs.add(task.spec());
+      }
+      this.specs = taskSpecs;
+    }
+
+    return specs;
+  }
+
+  @SuppressWarnings("unchecked")
+  protected synchronized List<T> tasks() {

Review Comment:
   Made `tasks` and `taskGroups` synchronized to follow `SparkCopyOnWriteScan` that consumes this logic now.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041613743


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java:
##########
@@ -366,9 +365,9 @@ public Scan buildCopyOnWriteScan() {
 
     Schema expectedSchema = schemaWithMetadataColumns();
 
-    TableScan scan =
+    BatchScan scan =

Review Comment:
   Had to change this so that `SparkCopyOnWriteScan` can use the same task planning.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042664536


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   You are right if would only be beneficial if the keys are compatible on both sides. What about scenarios when I manually distribute the incoming relation using Iceberg functions? It still does not work right now as the other side is not `KeyGroupPartitioning`.



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042661088


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   Yeah I really would want to be able to do something where the smaller side of the join is repartitioned using the transforms of the larger side if available. 



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042649365


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   @sunchao, how hard will it be to add support for distributing the smaller relation using the reported partitioning by an Iceberg table for Spark 3.4? I feel that's a bid deal.



##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   @sunchao, how hard will it be to add support for distributing the smaller relation using the reported partitioning by an Iceberg table for Spark 3.4? I feel that's a big deal.



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


[GitHub] [iceberg] sunchao commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042716054


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   > @sunchao, we can't use the newly added logic for pushing down common keys cause we don't know the keys in the relation we will shuffle (until we actually shuffle it)?
   
   Correct. We don't know the keys until runtime, which is too late.
   
   > I'd be great to at least support cases when users manually repartition the incoming source relation using the Iceberg function catalog. I believe that would produce HashPartitioning or RangePartitioning on the other side.
   
   Let me keep a note, and do some experiment on these ideas. 



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053563618


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java:
##########
@@ -143,4 +148,32 @@ public synchronized void testDeleteWithConcurrentTableRefresh() throws Exception
     executorService.shutdown();
     Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES));
   }
+
+  @Test
+  public void testRuntimeFilteringWithReportedPartitioning() throws NoSuchTableException {
+    createAndInitPartitionedTable();
+
+    append(new Employee(1, "hr"), new Employee(3, "hr"));
+    append(new Employee(1, "hardware"), new Employee(2, "hardware"));
+
+    Map<String, String> sqlConf =
+        ImmutableMap.of(
+            SQLConf.V2_BUCKETING_ENABLED().key(),
+            "true",
+            SparkSQLProperties.PRESERVE_DATA_GROUPING,
+            "true");
+
+    withSQLConf(sqlConf, () -> sql("DELETE FROM %s WHERE id = 2", tableName));
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots()));
+
+    Snapshot currentSnapshot = table.currentSnapshot();
+    validateCopyOnWrite(currentSnapshot, "1", "1", "1");

Review Comment:
   This is what we did historically to simplify the validation as we compare summary map (i.e. strings).
   I'll check.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053629122


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType = computeGroupingKeyType();
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private StructType computeGroupingKeyType() {

Review Comment:
   This method only exists to avoid Spotless formatting that splits this into multiple lines.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1052663495


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java:
##########
@@ -147,9 +152,9 @@ protected Statistics estimateStatistics(Snapshot snapshot) {
 
   @Override
   public String description() {
-    String filters =
-        filterExpressions.stream().map(Spark3Util::describe).collect(Collectors.joining(", "));
-    return String.format("%s [filters=%s]", table, filters);
+    return String.format(
+        "%s [filters=%s, groupedBy=%s]",
+        table(), Spark3Util.describe(filterExpressions), groupingKeyType());

Review Comment:
   This will be part of Spark UI.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1056852535


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java:
##########
@@ -147,9 +152,9 @@ protected Statistics estimateStatistics(Snapshot snapshot) {
 
   @Override
   public String description() {
-    String filters =
-        filterExpressions.stream().map(Spark3Util::describe).collect(Collectors.joining(", "));
-    return String.format("%s [filters=%s]", table, filters);
+    return String.format(
+        "%s [filters=%s, groupedBy=%s]",
+        table(), Spark3Util.describe(filterExpressions), groupingKeyType());

Review Comment:
   Switched to column names only.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055860143


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =
+      "spark.sql.iceberg.split.preserve-data-grouping";

Review Comment:
   Do we want "split" here? Would `preserve-data-grouping` be unclear? If not, what about `planning`?



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053523804


##########
core/src/main/java/org/apache/iceberg/Partitioning.java:
##########
@@ -215,11 +225,15 @@ public Void alwaysNull(int fieldId, String sourceName, int sourceId) {
    * that have the same field ID but use a void transform under the hood. Such fields cannot be part
    * of the grouping key as void transforms always return null.
    *
+   * <p>If the provided schema is not null, this method will only take into account partition fields
+   * on top of columns present in the schema. Otherwise, all partition fields will be considered.
+   *
+   * @param schema a schema specifying a set of source columns to consider (null to consider all)
    * @param specs one or many specs
    * @return the constructed grouping key type
    */
-  public static StructType groupingKeyType(Collection<PartitionSpec> specs) {
-    return buildPartitionProjectionType("grouping key", specs, commonActiveFieldIds(specs));
+  public static StructType groupingKeyType(Schema schema, Collection<PartitionSpec> specs) {
+    return buildPartitionProjectionType("grouping key", specs, commonActiveFieldIds(schema, specs));

Review Comment:
   How do we limit the specs passed to this method to just the ones that are used by manifests that are scanned during planning? (I may answer this myself later, but I want to write the question down)



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053564525


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java:
##########
@@ -140,4 +144,35 @@ public synchronized void testUpdateWithConcurrentTableRefresh() throws Exception
     executorService.shutdown();
     Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES));
   }
+
+  @Test
+  public void testRuntimeFilteringWithReportedPartitioning() {
+    createAndInitTable("id INT, dep STRING");
+    sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName);
+
+    append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 3, \"dep\": \"hr\" }");
+    append(
+        tableName,
+        "{ \"id\": 1, \"dep\": \"hardware\" }\n" + "{ \"id\": 2, \"dep\": \"hardware\" }");
+
+    Map<String, String> sqlConf =
+        ImmutableMap.of(
+            SQLConf.V2_BUCKETING_ENABLED().key(),
+            "true",
+            SparkSQLProperties.PRESERVE_DATA_GROUPING,
+            "true");
+
+    withSQLConf(sqlConf, () -> sql("UPDATE %s SET id = cast('-1' AS INT) WHERE id = 2", tableName));

Review Comment:
   Just copied it from another test. Will remove.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053558520


##########
core/src/main/java/org/apache/iceberg/Partitioning.java:
##########
@@ -200,6 +200,16 @@ public Void alwaysNull(int fieldId, String sourceName, int sourceId) {
   /**
    * Builds a grouping key type considering all provided specs.
    *
+   * @param specs one or many specs
+   * @return the constructed grouping key type
+   */
+  public static StructType groupingKeyType(Collection<PartitionSpec> specs) {

Review Comment:
   We could, I wasn't sure. If you have a set of specs, we probably have a schema too. I'll deprecate.



##########
core/src/main/java/org/apache/iceberg/Partitioning.java:
##########
@@ -200,6 +200,16 @@ public Void alwaysNull(int fieldId, String sourceName, int sourceId) {
   /**
    * Builds a grouping key type considering all provided specs.
    *
+   * @param specs one or many specs
+   * @return the constructed grouping key type
+   */
+  public static StructType groupingKeyType(Collection<PartitionSpec> specs) {

Review Comment:
   We could, I wasn't sure. If you have a set of specs, you probably have a schema too. I'll deprecate.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053562414


##########
core/src/test/java/org/apache/iceberg/TestPartitioning.java:
##########
@@ -354,6 +354,19 @@ public void testGroupingKeyTypeWithEvolvedUnpartitionedSpec() {
     Assert.assertEquals("Types must match", expectedType, actualType);
   }
 
+  @Test
+  public void testGroupingKeyTypeWithProjectedSchema() {
+    TestTables.TestTable table =
+        TestTables.create(tableDir, "test", SCHEMA, BY_CATEGORY_DATA_SPEC, V1_FORMAT_VERSION);
+
+    Schema projectedSchema = table.schema().select("id", "data");

Review Comment:
   Correct. Otherwise, we will break Spark because we can report non-projected columns.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1056862209


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType = computeGroupingKeyType();
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private StructType computeGroupingKeyType() {
+    return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    if (groupingKeyTransforms == null) {
+      List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+      Set<Integer> seenFieldIds = Sets.newHashSet();
+
+      for (PartitionSpec spec : specs()) {
+        for (PartitionField field : spec.fields()) {
+          int fieldId = field.fieldId();
+
+          if (groupingKeyType().containsField(fieldId) && !seenFieldIds.contains(fieldId)) {
+            groupingKeyFields.add(field);
+          }
+
+          seenFieldIds.add(fieldId);
+        }
+      }
+
+      this.groupingKeyTransforms = Spark3Util.toTransforms(table().schema(), groupingKeyFields);
+    }
+
+    return groupingKeyTransforms;
+  }
+
+  protected Set<PartitionSpec> specs() {
+    if (specs == null) {
+      Set<PartitionSpec> taskSpecs = Sets.newHashSet();
+      for (T task : tasks()) {
+        taskSpecs.add(task.spec());
+      }
+      this.specs = taskSpecs;
+    }
+
+    return specs;
+  }
+
+  protected synchronized List<T> tasks() {
+    if (tasks == null) {
+      try (CloseableIterable<? extends ScanTask> taskIterable = scan.planFiles()) {
+        List<T> plannedTasks = Lists.newArrayList();

Review Comment:
   I agree with you but I am not sure it applies here. We eagerly loaded tasks from day 1. They are used in lots of places and will need to be materialized immediately to compute stats, determine grouping key, etc.
   
   I think we will be able to gc filtered out tasks in the current implementation as well.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042643130


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   We will definitely need to support distributing the other side using the Iceberg function catalog. A common use case for this is MERGE. The incoming relation may be a view, which will never be distributed in a compatible way with Iceberg tables. I hope Spark would be smart enough to shuffle just the smaller relation using the reported partitioning from the target table.
   
   Unfortunately, it does not work even if I distribute the incoming data manually using the Iceberg function catalog. We must have `KeyGroupPartitioning` on both sides, which can only be reported by data sources.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042627234


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java:
##########
@@ -255,74 +256,90 @@ public static org.apache.iceberg.Table toIcebergTable(Table table) {
     return sparkTable.table();
   }
 
+  public static Transform[] toTransforms(Schema schema, List<PartitionField> fields) {
+    SpecTransformToSparkTransform visitor = new SpecTransformToSparkTransform(schema);
+
+    List<Transform> transforms = Lists.newArrayList();
+
+    for (PartitionField field : fields) {
+      Transform transform = PartitionSpecVisitor.visit(schema, field, visitor);
+      if (transform != null) {
+        transforms.add(transform);
+      }
+    }
+
+    return transforms.toArray(new Transform[0]);
+  }
+
   /**
    * Converts a PartitionSpec to Spark transforms.
    *
    * @param spec a PartitionSpec
    * @return an array of Transforms
    */
   public static Transform[] toTransforms(PartitionSpec spec) {
-    Map<Integer, String> quotedNameById = SparkSchemaUtil.indexQuotedNameById(spec.schema());
-    List<Transform> transforms =
-        PartitionSpecVisitor.visit(
-            spec,
-            new PartitionSpecVisitor<Transform>() {
-              @Override
-              public Transform identity(String sourceName, int sourceId) {
-                return Expressions.identity(quotedName(sourceId));
-              }
+    SpecTransformToSparkTransform visitor = new SpecTransformToSparkTransform(spec.schema());
+    List<Transform> transforms = PartitionSpecVisitor.visit(spec, visitor);
+    return transforms.stream().filter(Objects::nonNull).toArray(Transform[]::new);
+  }
 
-              @Override
-              public Transform bucket(String sourceName, int sourceId, int numBuckets) {
-                return Expressions.bucket(numBuckets, quotedName(sourceId));
-              }
+  private static class SpecTransformToSparkTransform implements PartitionSpecVisitor<Transform> {
+    private final Map<Integer, String> quotedNameById;
 
-              @Override
-              public Transform truncate(String sourceName, int sourceId, int width) {
-                return Expressions.apply(
-                    "truncate",
-                    Expressions.column(quotedName(sourceId)),
-                    Expressions.literal(width));
-              }
+    SpecTransformToSparkTransform(Schema schema) {
+      this.quotedNameById = SparkSchemaUtil.indexQuotedNameById(schema);
+    }
 
-              @Override
-              public Transform year(String sourceName, int sourceId) {
-                return Expressions.years(quotedName(sourceId));
-              }
+    @Override
+    public Transform identity(String sourceName, int sourceId) {
+      return Expressions.identity(quotedName(sourceId));
+    }
 
-              @Override
-              public Transform month(String sourceName, int sourceId) {
-                return Expressions.months(quotedName(sourceId));
-              }
+    @Override
+    public Transform bucket(String sourceName, int sourceId, int numBuckets) {
+      return Expressions.bucket(numBuckets, quotedName(sourceId));
+    }
 
-              @Override
-              public Transform day(String sourceName, int sourceId) {
-                return Expressions.days(quotedName(sourceId));
-              }
+    @Override
+    public Transform truncate(String sourceName, int sourceId, int width) {
+      NamedReference column = Expressions.column(quotedName(sourceId));
+      return Expressions.apply("truncate", Expressions.literal(width), column);

Review Comment:
   I think the only place that would change is string output of partitioning in `SparkTable`. Otherwise, we handle both combinations in the `TruncateTransform` extractor.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042622671


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =

Review Comment:
   I am afraid this one will be pretty public. Users will have to explicitly enable this as we don't know if Spark can benefit from the reported distribution and skip shuffles. Hence, we disable it by default to avoid any performance regressions caused by less dense packing of splits.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042708006


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   @sunchao, we can't use the newly added logic for pushing down common keys cause we don't know the keys in the relation we will shuffle (until we actually shuffle it)? 



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042708006


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   @sunchao, we can't use the newly added logic for pushing down common keys cause we don't know the keys in the relation we will shuffle? 



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041609274


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java:
##########
@@ -255,74 +256,90 @@ public static org.apache.iceberg.Table toIcebergTable(Table table) {
     return sparkTable.table();
   }
 
+  public static Transform[] toTransforms(Schema schema, List<PartitionField> fields) {
+    SpecTransformToSparkTransform visitor = new SpecTransformToSparkTransform(schema);

Review Comment:
   I pulled the anonymous class below into `SpecTransformToSparkTransform` to reuse 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-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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041611507


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =

Review Comment:
   There were multiple names I considered and `spark.sql.iceberg.split.preserve-partition-boundaries` was one of them. I discarded it because we are not really preserving partition boundaries if there are multiple specs. Also, Spark will push down join keys in the future so we won't really respect partition boundaries as such.
   
   That being said, I'll think more tomorrow.
   
   



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041615901


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java:
##########
@@ -257,6 +268,42 @@ protected Dataset<Row> jsonToDF(String schema, String... records) {
     return spark.read().schema(schema).json(jsonDF);
   }
 
+  protected SparkPlan executeAndKeepPlan(String query, Object... args) {
+    return executeAndKeepPlan(() -> sql(query, args));
+  }
+
+  protected SparkPlan executeAndKeepPlan(Action action) {

Review Comment:
   A utility method to fetch what `SparkPlan` was actually executed.



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042413822


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   Should we have some tests with multiple buckets? or multiple partition transforms? 



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042649365


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   @sunchao, how hard will it be to add support for distributing the smaller relation using the reported partitioning by an Iceberg table for Spark 3.4?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042644906


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(

Review Comment:
   I'll add this to the notes.



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042691741


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   It uses Inner Join for merges without a "when not matching" 
   
   I would think for outer join we would similarly just keep all of the non-matching partitions separate but I haven't really looked at the code



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1052493128


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    Preconditions.checkState(taskGroups() != null, "Task groups must be planned");
+
+    if (groupingKeyType().fields().isEmpty()) {
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType =
+            org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    Set<Integer> groupingKeyFieldIds =
+        groupingKeyType().fields().stream()
+            .map(Types.NestedField::fieldId)
+            .collect(Collectors.toSet());
+
+    List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+    for (PartitionSpec spec : specs()) {
+      for (PartitionField field : spec.fields()) {
+        if (groupingKeyFieldIds.contains(field.fieldId())) {
+          groupingKeyFields.add(field);
+          groupingKeyFieldIds.remove(field.fieldId());
+        }
+      }
+    }
+
+    return Spark3Util.toTransforms(table().schema(), groupingKeyFields);
+  }
+
+  protected Set<PartitionSpec> specs() {
+    if (specs == null) {
+      Set<PartitionSpec> taskSpecs = Sets.newHashSet();
+      for (T task : tasks()) {
+        taskSpecs.add(task.spec());
+      }
+      this.specs = taskSpecs;
+    }
+
+    return specs;
+  }
+
+  @SuppressWarnings("unchecked")
+  protected synchronized List<T> tasks() {
+    if (tasks == null) {
+      try (CloseableIterable<? extends ScanTask> taskIterable = scan.planFiles()) {
+        List<T> plannedTasks = Lists.newArrayList();
+
+        for (ScanTask task : taskIterable) {
+          ValidationException.check(

Review Comment:
   This check is to provide a reasonable exception if there is a task of an unsupported type. Since we may have a combination of different tasks, each task is being validated.
   
   I don't think it would be super expensive to perform this validation but it is kind of optional too.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1056852915


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =
+      "spark.sql.iceberg.split.preserve-data-grouping";

Review Comment:
   Switched to `planning`.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1056852815


##########
core/src/main/java/org/apache/iceberg/Partitioning.java:
##########
@@ -200,6 +200,16 @@ public Void alwaysNull(int fieldId, String sourceName, int sourceId) {
   /**
    * Builds a grouping key type considering all provided specs.
    *
+   * @param specs one or many specs
+   * @return the constructed grouping key type
+   */
+  public static StructType groupingKeyType(Collection<PartitionSpec> specs) {

Review Comment:
   Deprecated.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055955844


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType = computeGroupingKeyType();
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private StructType computeGroupingKeyType() {
+    return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    if (groupingKeyTransforms == null) {
+      List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+      Set<Integer> seenFieldIds = Sets.newHashSet();
+
+      for (PartitionSpec spec : specs()) {
+        for (PartitionField field : spec.fields()) {
+          int fieldId = field.fieldId();
+
+          if (groupingKeyType().containsField(fieldId) && !seenFieldIds.contains(fieldId)) {
+            groupingKeyFields.add(field);
+          }
+
+          seenFieldIds.add(fieldId);
+        }
+      }
+
+      this.groupingKeyTransforms = Spark3Util.toTransforms(table().schema(), groupingKeyFields);
+    }
+
+    return groupingKeyTransforms;
+  }
+
+  protected Set<PartitionSpec> specs() {
+    if (specs == null) {
+      Set<PartitionSpec> taskSpecs = Sets.newHashSet();
+      for (T task : tasks()) {
+        taskSpecs.add(task.spec());
+      }
+      this.specs = taskSpecs;
+    }
+
+    return specs;
+  }
+
+  protected synchronized List<T> tasks() {
+    if (tasks == null) {
+      try (CloseableIterable<? extends ScanTask> taskIterable = scan.planFiles()) {
+        List<T> plannedTasks = Lists.newArrayList();
+
+        for (ScanTask task : taskIterable) {
+          ValidationException.check(
+              taskJavaClass().isInstance(task),
+              "Unsupported task type, expected a subtype of %s: %",
+              taskJavaClass().getName(),
+              task.getClass().getName());
+
+          plannedTasks.add(taskJavaClass().cast(task));
+        }
+
+        LOG.debug("Planned {} tasks", plannedTasks.size());
+
+        this.tasks = plannedTasks;
+      } catch (IOException e) {
+        throw new UncheckedIOException("Failed to close scan: " + scan, e);
+      }
+    }
+
+    return tasks;
+  }
+
+  @Override
+  protected synchronized List<ScanTaskGroup<T>> taskGroups() {
+    if (taskGroups == null) {
+      if (groupingKeyType().fields().isEmpty()) {
+        CloseableIterable<ScanTaskGroup<T>> plannedTaskGroups =
+            TableScanUtil.planTaskGroups(
+                CloseableIterable.withNoopClose(tasks()),
+                scan.targetSplitSize(),
+                scan.splitLookback(),
+                scan.splitOpenFileCost());
+        this.taskGroups = Lists.newArrayList(plannedTaskGroups);
+
+        LOG.debug("Planned {} task group(s) without data grouping", taskGroups.size());
+
+      } else {
+        List<ScanTaskGroup<T>> plannedTaskGroups =
+            TableScanUtil.planTaskGroups(
+                tasks(),
+                scan.targetSplitSize(),
+                scan.splitLookback(),
+                scan.splitOpenFileCost(),
+                groupingKeyType());
+        StructLikeSet plannedGroupingKeys = groupingKeys(plannedTaskGroups);
+
+        LOG.debug(

Review Comment:
   Is this useful? Maybe the number of unique grouping keys and the type.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055955191


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType = computeGroupingKeyType();
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private StructType computeGroupingKeyType() {
+    return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    if (groupingKeyTransforms == null) {
+      List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+      Set<Integer> seenFieldIds = Sets.newHashSet();
+
+      for (PartitionSpec spec : specs()) {
+        for (PartitionField field : spec.fields()) {
+          int fieldId = field.fieldId();
+
+          if (groupingKeyType().containsField(fieldId) && !seenFieldIds.contains(fieldId)) {
+            groupingKeyFields.add(field);
+          }
+
+          seenFieldIds.add(fieldId);
+        }
+      }
+
+      this.groupingKeyTransforms = Spark3Util.toTransforms(table().schema(), groupingKeyFields);
+    }
+
+    return groupingKeyTransforms;
+  }
+
+  protected Set<PartitionSpec> specs() {
+    if (specs == null) {
+      Set<PartitionSpec> taskSpecs = Sets.newHashSet();
+      for (T task : tasks()) {
+        taskSpecs.add(task.spec());
+      }
+      this.specs = taskSpecs;
+    }
+
+    return specs;
+  }
+
+  protected synchronized List<T> tasks() {
+    if (tasks == null) {
+      try (CloseableIterable<? extends ScanTask> taskIterable = scan.planFiles()) {
+        List<T> plannedTasks = Lists.newArrayList();

Review Comment:
   Minor: It looks like this was already done in `SparkBatchQueryScan`, but I generally prefer to keep things as a `CloseableIterable` as long as possible. This could probably return a `CloseableIterable` instead.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1054947098


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =

Review Comment:
   I think this name makes sense.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053540842


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java:
##########
@@ -140,4 +144,35 @@ public synchronized void testUpdateWithConcurrentTableRefresh() throws Exception
     executorService.shutdown();
     Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES));
   }
+
+  @Test
+  public void testRuntimeFilteringWithReportedPartitioning() {
+    createAndInitTable("id INT, dep STRING");
+    sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName);
+
+    append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 3, \"dep\": \"hr\" }");
+    append(
+        tableName,
+        "{ \"id\": 1, \"dep\": \"hardware\" }\n" + "{ \"id\": 2, \"dep\": \"hardware\" }");
+
+    Map<String, String> sqlConf =
+        ImmutableMap.of(
+            SQLConf.V2_BUCKETING_ENABLED().key(),
+            "true",
+            SparkSQLProperties.PRESERVE_DATA_GROUPING,
+            "true");
+
+    withSQLConf(sqlConf, () -> sql("UPDATE %s SET id = cast('-1' AS INT) WHERE id = 2", tableName));

Review Comment:
   Why is the cast needed?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053570254


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,586 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {

Review Comment:
   @rdblue, this is the primary test suite that covers different transforms. I had just a few tests for row-level operations to make sure it works there too.



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


[GitHub] [iceberg] aokolnychyi merged pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi merged PR #6371:
URL: https://github.com/apache/iceberg/pull/6371


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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055945015


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java:
##########
@@ -147,9 +152,9 @@ protected Statistics estimateStatistics(Snapshot snapshot) {
 
   @Override
   public String description() {
-    String filters =
-        filterExpressions.stream().map(Spark3Util::describe).collect(Collectors.joining(", "));
-    return String.format("%s [filters=%s]", table, filters);
+    return String.format(
+        "%s [filters=%s, groupedBy=%s]",
+        table(), Spark3Util.describe(filterExpressions), groupingKeyType());

Review Comment:
   Do we want a simpler grouping expression? This is going to show `struct<...>` where we may just want the column names from that struct.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1056852402


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java:
##########
@@ -356,4 +357,23 @@ private <T> GenericArrayData fillArray(
 
     return new GenericArrayData(array);
   }
+
+  @Override
+  public boolean equals(Object other) {

Review Comment:
   It is a bug in Spark that will be fixed. Spark uses `groupBy` in one place, which relies on `equals`.
   All other places use `InternalRowSet`.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1056852878


##########
api/src/main/java/org/apache/iceberg/types/Types.java:
##########
@@ -554,6 +554,10 @@ public List<NestedField> fields() {
       return lazyFieldList();
     }
 
+    public boolean containsField(int id) {

Review Comment:
   Rewrote the other part and removed this.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1056022196


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java:
##########
@@ -143,4 +148,32 @@ public synchronized void testDeleteWithConcurrentTableRefresh() throws Exception
     executorService.shutdown();
     Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES));
   }
+
+  @Test
+  public void testRuntimeFilteringWithReportedPartitioning() throws NoSuchTableException {
+    createAndInitPartitionedTable();
+
+    append(new Employee(1, "hr"), new Employee(3, "hr"));
+    append(new Employee(1, "hardware"), new Employee(2, "hardware"));
+
+    Map<String, String> sqlConf =
+        ImmutableMap.of(
+            SQLConf.V2_BUCKETING_ENABLED().key(),
+            "true",
+            SparkSQLProperties.PRESERVE_DATA_GROUPING,
+            "true");
+
+    withSQLConf(sqlConf, () -> sql("DELETE FROM %s WHERE id = 2", tableName));
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots()));
+
+    Snapshot currentSnapshot = table.currentSnapshot();
+    validateCopyOnWrite(currentSnapshot, "1", "1", "1");

Review Comment:
   Doing this properly would touch lots of existing places. I'll follow up to fix this separately.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055879720


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =
+      "spark.sql.iceberg.split.preserve-data-grouping";

Review Comment:
   I was inspired by `read.split` table properties we have. If omitted, it is not clear whether it applies only to reads. I like `planning`. We could also go for `read.split` to follow table properties but either one is equally good for me.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041628267


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java:
##########
@@ -255,74 +256,90 @@ public static org.apache.iceberg.Table toIcebergTable(Table table) {
     return sparkTable.table();
   }
 
+  public static Transform[] toTransforms(Schema schema, List<PartitionField> fields) {
+    SpecTransformToSparkTransform visitor = new SpecTransformToSparkTransform(schema);
+
+    List<Transform> transforms = Lists.newArrayList();
+
+    for (PartitionField field : fields) {
+      Transform transform = PartitionSpecVisitor.visit(schema, field, visitor);
+      if (transform != null) {
+        transforms.add(transform);
+      }
+    }
+
+    return transforms.toArray(new Transform[0]);
+  }
+
   /**
    * Converts a PartitionSpec to Spark transforms.
    *
    * @param spec a PartitionSpec
    * @return an array of Transforms
    */
   public static Transform[] toTransforms(PartitionSpec spec) {
-    Map<Integer, String> quotedNameById = SparkSchemaUtil.indexQuotedNameById(spec.schema());
-    List<Transform> transforms =
-        PartitionSpecVisitor.visit(
-            spec,
-            new PartitionSpecVisitor<Transform>() {
-              @Override
-              public Transform identity(String sourceName, int sourceId) {
-                return Expressions.identity(quotedName(sourceId));
-              }
+    SpecTransformToSparkTransform visitor = new SpecTransformToSparkTransform(spec.schema());
+    List<Transform> transforms = PartitionSpecVisitor.visit(spec, visitor);
+    return transforms.stream().filter(Objects::nonNull).toArray(Transform[]::new);
+  }
 
-              @Override
-              public Transform bucket(String sourceName, int sourceId, int numBuckets) {
-                return Expressions.bucket(numBuckets, quotedName(sourceId));
-              }
+  private static class SpecTransformToSparkTransform implements PartitionSpecVisitor<Transform> {
+    private final Map<Integer, String> quotedNameById;
 
-              @Override
-              public Transform truncate(String sourceName, int sourceId, int width) {
-                return Expressions.apply(
-                    "truncate",
-                    Expressions.column(quotedName(sourceId)),
-                    Expressions.literal(width));
-              }
+    SpecTransformToSparkTransform(Schema schema) {
+      this.quotedNameById = SparkSchemaUtil.indexQuotedNameById(schema);
+    }
 
-              @Override
-              public Transform year(String sourceName, int sourceId) {
-                return Expressions.years(quotedName(sourceId));
-              }
+    @Override
+    public Transform identity(String sourceName, int sourceId) {
+      return Expressions.identity(quotedName(sourceId));
+    }
 
-              @Override
-              public Transform month(String sourceName, int sourceId) {
-                return Expressions.months(quotedName(sourceId));
-              }
+    @Override
+    public Transform bucket(String sourceName, int sourceId, int numBuckets) {
+      return Expressions.bucket(numBuckets, quotedName(sourceId));
+    }
 
-              @Override
-              public Transform day(String sourceName, int sourceId) {
-                return Expressions.days(quotedName(sourceId));
-              }
+    @Override
+    public Transform truncate(String sourceName, int sourceId, int width) {
+      NamedReference column = Expressions.column(quotedName(sourceId));
+      return Expressions.apply("truncate", Expressions.literal(width), column);

Review Comment:
   Passing `width` first to match `TruncateFunction`. Otherwise, the function catalog won't resolve.



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


[GitHub] [iceberg] sunchao commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042551459


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(

Review Comment:
   I think only `SQLConf.V2_BUCKETING_ENABLED` and `SparkSQLProperties.PRESERVE_DATA_GROUPING` are required, and the others are just for the convenience of testing.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042632953


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    Preconditions.checkState(taskGroups() != null, "Task groups must be planned");
+
+    if (groupingKeyType().fields().isEmpty()) {
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());

Review Comment:
   I'll add logs for now.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042631790


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =
+      "spark.sql.iceberg.split.preserve-data-grouping";
+  public static final boolean PRESERVE_DATA_GROUPING_DEFAULT = false;

Review Comment:
   My worry is the performance regressions that this may cause. There may be substantially more splits if this config is on. In order to benefit from SPJ, joins must have equality conditions on partition columns. Spark will propagate join conditions in the future. Our long-term plan may be to check if v2 bucketing enabled and whether we have join conditions on partition columns to return true by default. Even that will be suboptimal cause we don't know if SPJ would work.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =
+      "spark.sql.iceberg.split.preserve-data-grouping";
+  public static final boolean PRESERVE_DATA_GROUPING_DEFAULT = false;

Review Comment:
   My worry is performance regressions that this may cause. There may be substantially more splits if this config is on. In order to benefit from SPJ, joins must have equality conditions on partition columns. Spark will propagate join conditions in the future. Our long-term plan may be to check if v2 bucketing enabled and whether we have join conditions on partition columns to return true by default. Even that will be suboptimal cause we don't know if SPJ would work.



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


[GitHub] [iceberg] sunchao commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042705070


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   Maybe we could just distribute all the non-matching rows "randomly" into the N partitions, but I'll need to check more closely on Spark side. It's a bit complicated. 



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042357442


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =
+      "spark.sql.iceberg.split.preserve-data-grouping";
+  public static final boolean PRESERVE_DATA_GROUPING_DEFAULT = false;

Review Comment:
   My question here would be should we really have the default be false? I have no problem with the name but It feels like this is probably always the right decision for a scan with possible joins. We should probably look into this more in the future but my guess is that if a query requires partition columns, we should group by those columns.
   
   That's my long way to say i'm fine with the default being false for now, but I think it should probably be true.



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


[GitHub] [iceberg] zinking commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
zinking commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1045716950


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    Preconditions.checkState(taskGroups() != null, "Task groups must be planned");
+
+    if (groupingKeyType().fields().isEmpty()) {
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType =
+            org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    Set<Integer> groupingKeyFieldIds =
+        groupingKeyType().fields().stream()
+            .map(Types.NestedField::fieldId)
+            .collect(Collectors.toSet());
+
+    List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+    for (PartitionSpec spec : specs()) {
+      for (PartitionField field : spec.fields()) {
+        if (groupingKeyFieldIds.contains(field.fieldId())) {
+          groupingKeyFields.add(field);
+          groupingKeyFieldIds.remove(field.fieldId());
+        }
+      }
+    }
+
+    return Spark3Util.toTransforms(table().schema(), groupingKeyFields);
+  }
+
+  protected Set<PartitionSpec> specs() {
+    if (specs == null) {
+      Set<PartitionSpec> taskSpecs = Sets.newHashSet();
+      for (T task : tasks()) {
+        taskSpecs.add(task.spec());
+      }
+      this.specs = taskSpecs;
+    }
+
+    return specs;
+  }
+
+  @SuppressWarnings("unchecked")
+  protected synchronized List<T> tasks() {
+    if (tasks == null) {
+      try (CloseableIterable<? extends ScanTask> taskIterable = scan.planFiles()) {
+        List<T> plannedTasks = Lists.newArrayList();
+
+        for (ScanTask task : taskIterable) {
+          ValidationException.check(

Review Comment:
   is this desirable to do the validation in the for LOOP? the exception will be thrown anyway.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055860782


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =
+      "spark.sql.iceberg.split.preserve-data-grouping";
+  public static final boolean PRESERVE_DATA_GROUPING_DEFAULT = false;

Review Comment:
   I think long term we will want to get rid of this, but that will require some help from Spark. Ideally, Spark should tell the source whether or not it cares about preserving grouping, and on which columns it matters. If we had that information we wouldn't need this at all.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053566628


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.extensions;
+
+import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE;
+import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ;
+
+import java.util.Map;
+import org.apache.iceberg.RowLevelOperationMode;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.spark.sql.execution.SparkPlan;
+import org.apache.spark.sql.internal.SQLConf;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+public class TestStoragePartitionedJoinsInRowLevelOperations extends SparkExtensionsTestBase {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final Map<String, String> COMMON_TABLE_PROPERTIES =
+      ImmutableMap.of(
+          TableProperties.FORMAT_VERSION,
+          "2",
+          TableProperties.SPLIT_SIZE,
+          "16777216",
+          TableProperties.SPLIT_OPEN_FILE_COST,
+          "16777216");
+
+  // only v2 bucketing and preserve data grouping properties have to be enabled to trigger SPJ
+  // other properties are only to simplify testing and validation
+  private static final Map<String, String> ENABLED_SPJ_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {
+        SparkCatalogConfig.HIVE.catalogName(),
+        SparkCatalogConfig.HIVE.implementation(),
+        SparkCatalogConfig.HIVE.properties()
+      }
+    };
+  }
+
+  public TestStoragePartitionedJoinsInRowLevelOperations(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  @Test
+  public void testCopyOnWriteDeleteWithoutShuffles() {
+    checkDelete(COPY_ON_WRITE);
+  }
+
+  @Test
+  public void testMergeOnReadDeleteWithoutShuffles() {
+    checkDelete(MERGE_ON_READ);
+  }
+
+  private void checkDelete(RowLevelOperationMode mode) {
+    String createTableStmt =
+        "CREATE TABLE %s (id INT, salary INT, dep STRING)"
+            + "USING iceberg "
+            + "PARTITIONED BY (dep) "
+            + "TBLPROPERTIES (%s)";
+
+    sql(createTableStmt, tableName, tablePropsAsString(COMMON_TABLE_PROPERTIES));
+
+    append(tableName, "{ \"id\": 1, \"salary\": 100, \"dep\": \"hr\" }");
+    append(tableName, "{ \"id\": 2, \"salary\": 200, \"dep\": \"hr\" }");
+    append(tableName, "{ \"id\": 3, \"salary\": 300, \"dep\": \"hr\" }");
+    append(tableName, "{ \"id\": 4, \"salary\": 400, \"dep\": \"hardware\" }");
+
+    sql(createTableStmt, tableName(OTHER_TABLE_NAME), tablePropsAsString(COMMON_TABLE_PROPERTIES));
+
+    append(tableName(OTHER_TABLE_NAME), "{ \"id\": 1, \"salary\": 110, \"dep\": \"hr\" }");
+    append(tableName(OTHER_TABLE_NAME), "{ \"id\": 5, \"salary\": 500, \"dep\": \"hr\" }");
+
+    Map<String, String> deleteTableProps =
+        ImmutableMap.of(
+            TableProperties.DELETE_MODE,
+            mode.modeName(),
+            TableProperties.DELETE_DISTRIBUTION_MODE,
+            "none");
+
+    sql("ALTER TABLE %s SET TBLPROPERTIES(%s)", tableName, tablePropsAsString(deleteTableProps));
+
+    withSQLConf(
+        ENABLED_SPJ_SQL_CONF,
+        () -> {
+          SparkPlan plan =
+              executeAndKeepPlan(
+                  "DELETE FROM %s t WHERE "
+                      + "EXISTS (SELECT 1 FROM %s s WHERE t.id = s.id AND t.dep = s.dep) AND "
+                      + "dep = 'hr'",

Review Comment:
   SPJ requires the same set of keys on both sides. The delta relation contains only records in `hr`. If I skip this, SPJ would not apply. I make sure we don't have a single partition by configuring the split size and writing more than 1 file.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055955191


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType = computeGroupingKeyType();
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private StructType computeGroupingKeyType() {
+    return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    if (groupingKeyTransforms == null) {
+      List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+      Set<Integer> seenFieldIds = Sets.newHashSet();
+
+      for (PartitionSpec spec : specs()) {
+        for (PartitionField field : spec.fields()) {
+          int fieldId = field.fieldId();
+
+          if (groupingKeyType().containsField(fieldId) && !seenFieldIds.contains(fieldId)) {
+            groupingKeyFields.add(field);
+          }
+
+          seenFieldIds.add(fieldId);
+        }
+      }
+
+      this.groupingKeyTransforms = Spark3Util.toTransforms(table().schema(), groupingKeyFields);
+    }
+
+    return groupingKeyTransforms;
+  }
+
+  protected Set<PartitionSpec> specs() {
+    if (specs == null) {
+      Set<PartitionSpec> taskSpecs = Sets.newHashSet();
+      for (T task : tasks()) {
+        taskSpecs.add(task.spec());
+      }
+      this.specs = taskSpecs;
+    }
+
+    return specs;
+  }
+
+  protected synchronized List<T> tasks() {
+    if (tasks == null) {
+      try (CloseableIterable<? extends ScanTask> taskIterable = scan.planFiles()) {
+        List<T> plannedTasks = Lists.newArrayList();

Review Comment:
   Minor: It looks like this was already done in `SparkBatchQueryScan`, but I generally prefer to keep things as a `CloseableIterable` as long as possible. This could probably return a `CloseableIterable` instead. That might help if we want to actually forget about tasks and gc them after runtime filtering.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055951820


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType = computeGroupingKeyType();
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private StructType computeGroupingKeyType() {
+    return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    if (groupingKeyTransforms == null) {
+      List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+      Set<Integer> seenFieldIds = Sets.newHashSet();
+
+      for (PartitionSpec spec : specs()) {

Review Comment:
   @aokolnychyi, I don't quite understand why this doesn't just get the field IDs from `groupingKeyType()`. In order to add an ID to `groupingKeyFields`, the ID must be in `groupingKeyType()`. Using that type would also ensure that each ID is used only once, so there would be no need for `seenFieldIds`.
   
   The main reason for this that I can think of is that this depends on the field order from from looping through specs like this... but shouldn't these transforms actually match the order of `groupingKeyType`? Rather than depending on the order to match between the key type and the transforms here, I think it would make more sense to just build a map from partition field ID to partition field, then loop over the grouping key fields in order.



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


[GitHub] [iceberg] sunchao commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042689202


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   Does `MERGE INTO` only use inner-join? If so, we can start thinking about implementing this in Spark (however I'm not sure if it will make it into Spark 3.4). I briefly mentioned about the idea to the Spark community before, but they were concerned about adding `KeyGroupedPartitioning` to the shuffle mechanism. I feel there would be less objection if we introduce a flag to control the behavior though.



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042682299


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   Wouldn't that make it very easy ? If they don't belong to any partition on the iceberg side then we know there aren't any matches for those partitions?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041628267


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java:
##########
@@ -255,74 +256,90 @@ public static org.apache.iceberg.Table toIcebergTable(Table table) {
     return sparkTable.table();
   }
 
+  public static Transform[] toTransforms(Schema schema, List<PartitionField> fields) {
+    SpecTransformToSparkTransform visitor = new SpecTransformToSparkTransform(schema);
+
+    List<Transform> transforms = Lists.newArrayList();
+
+    for (PartitionField field : fields) {
+      Transform transform = PartitionSpecVisitor.visit(schema, field, visitor);
+      if (transform != null) {
+        transforms.add(transform);
+      }
+    }
+
+    return transforms.toArray(new Transform[0]);
+  }
+
   /**
    * Converts a PartitionSpec to Spark transforms.
    *
    * @param spec a PartitionSpec
    * @return an array of Transforms
    */
   public static Transform[] toTransforms(PartitionSpec spec) {
-    Map<Integer, String> quotedNameById = SparkSchemaUtil.indexQuotedNameById(spec.schema());
-    List<Transform> transforms =
-        PartitionSpecVisitor.visit(
-            spec,
-            new PartitionSpecVisitor<Transform>() {
-              @Override
-              public Transform identity(String sourceName, int sourceId) {
-                return Expressions.identity(quotedName(sourceId));
-              }
+    SpecTransformToSparkTransform visitor = new SpecTransformToSparkTransform(spec.schema());
+    List<Transform> transforms = PartitionSpecVisitor.visit(spec, visitor);
+    return transforms.stream().filter(Objects::nonNull).toArray(Transform[]::new);
+  }
 
-              @Override
-              public Transform bucket(String sourceName, int sourceId, int numBuckets) {
-                return Expressions.bucket(numBuckets, quotedName(sourceId));
-              }
+  private static class SpecTransformToSparkTransform implements PartitionSpecVisitor<Transform> {
+    private final Map<Integer, String> quotedNameById;
 
-              @Override
-              public Transform truncate(String sourceName, int sourceId, int width) {
-                return Expressions.apply(
-                    "truncate",
-                    Expressions.column(quotedName(sourceId)),
-                    Expressions.literal(width));
-              }
+    SpecTransformToSparkTransform(Schema schema) {
+      this.quotedNameById = SparkSchemaUtil.indexQuotedNameById(schema);
+    }
 
-              @Override
-              public Transform year(String sourceName, int sourceId) {
-                return Expressions.years(quotedName(sourceId));
-              }
+    @Override
+    public Transform identity(String sourceName, int sourceId) {
+      return Expressions.identity(quotedName(sourceId));
+    }
 
-              @Override
-              public Transform month(String sourceName, int sourceId) {
-                return Expressions.months(quotedName(sourceId));
-              }
+    @Override
+    public Transform bucket(String sourceName, int sourceId, int numBuckets) {
+      return Expressions.bucket(numBuckets, quotedName(sourceId));
+    }
 
-              @Override
-              public Transform day(String sourceName, int sourceId) {
-                return Expressions.days(quotedName(sourceId));
-              }
+    @Override
+    public Transform truncate(String sourceName, int sourceId, int width) {
+      NamedReference column = Expressions.column(quotedName(sourceId));
+      return Expressions.apply("truncate", Expressions.literal(width), column);

Review Comment:
   Passing `width` first to match `TruncateFunction`.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041629288


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =
+      "spark.sql.iceberg.split.preserve-data-grouping";
+  public static final boolean PRESERVE_DATA_GROUPING_DEFAULT = false;

Review Comment:
   The default is false to avoid introducing a performance regression as split planning may be less dense.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041626586


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java:
##########
@@ -404,15 +404,15 @@ public void testSparkTableAddDropPartitions() throws Exception {
     assertPartitioningEquals(sparkTable(), 1, "bucket(16, id)");
 
     sql("ALTER TABLE %s ADD PARTITION FIELD truncate(data, 4)", tableName);
-    assertPartitioningEquals(sparkTable(), 2, "truncate(data, 4)");
+    assertPartitioningEquals(sparkTable(), 2, "truncate(4, data)");

Review Comment:
   I had to change `Spark3Util.toTransforms` to match `TruncateFunction`, which expects the width first.



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042346944


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java:
##########
@@ -255,74 +256,90 @@ public static org.apache.iceberg.Table toIcebergTable(Table table) {
     return sparkTable.table();
   }
 
+  public static Transform[] toTransforms(Schema schema, List<PartitionField> fields) {
+    SpecTransformToSparkTransform visitor = new SpecTransformToSparkTransform(schema);
+
+    List<Transform> transforms = Lists.newArrayList();
+
+    for (PartitionField field : fields) {
+      Transform transform = PartitionSpecVisitor.visit(schema, field, visitor);
+      if (transform != null) {
+        transforms.add(transform);
+      }
+    }
+
+    return transforms.toArray(new Transform[0]);
+  }
+
   /**
    * Converts a PartitionSpec to Spark transforms.
    *
    * @param spec a PartitionSpec
    * @return an array of Transforms
    */
   public static Transform[] toTransforms(PartitionSpec spec) {
-    Map<Integer, String> quotedNameById = SparkSchemaUtil.indexQuotedNameById(spec.schema());
-    List<Transform> transforms =
-        PartitionSpecVisitor.visit(
-            spec,
-            new PartitionSpecVisitor<Transform>() {
-              @Override
-              public Transform identity(String sourceName, int sourceId) {
-                return Expressions.identity(quotedName(sourceId));
-              }
+    SpecTransformToSparkTransform visitor = new SpecTransformToSparkTransform(spec.schema());
+    List<Transform> transforms = PartitionSpecVisitor.visit(spec, visitor);
+    return transforms.stream().filter(Objects::nonNull).toArray(Transform[]::new);
+  }
 
-              @Override
-              public Transform bucket(String sourceName, int sourceId, int numBuckets) {
-                return Expressions.bucket(numBuckets, quotedName(sourceId));
-              }
+  private static class SpecTransformToSparkTransform implements PartitionSpecVisitor<Transform> {
+    private final Map<Integer, String> quotedNameById;
 
-              @Override
-              public Transform truncate(String sourceName, int sourceId, int width) {
-                return Expressions.apply(
-                    "truncate",
-                    Expressions.column(quotedName(sourceId)),
-                    Expressions.literal(width));
-              }
+    SpecTransformToSparkTransform(Schema schema) {
+      this.quotedNameById = SparkSchemaUtil.indexQuotedNameById(schema);
+    }
 
-              @Override
-              public Transform year(String sourceName, int sourceId) {
-                return Expressions.years(quotedName(sourceId));
-              }
+    @Override
+    public Transform identity(String sourceName, int sourceId) {
+      return Expressions.identity(quotedName(sourceId));
+    }
 
-              @Override
-              public Transform month(String sourceName, int sourceId) {
-                return Expressions.months(quotedName(sourceId));
-              }
+    @Override
+    public Transform bucket(String sourceName, int sourceId, int numBuckets) {
+      return Expressions.bucket(numBuckets, quotedName(sourceId));
+    }
 
-              @Override
-              public Transform day(String sourceName, int sourceId) {
-                return Expressions.days(quotedName(sourceId));
-              }
+    @Override
+    public Transform truncate(String sourceName, int sourceId, int width) {
+      NamedReference column = Expressions.column(quotedName(sourceId));
+      return Expressions.apply("truncate", Expressions.literal(width), column);

Review Comment:
   So this is ok as a change because the previous version wouldn't have actually worked correct? Just making sure we aren't breaking any api 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-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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042664536


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   You are right if would only be beneficial if the keys are compatible on both sides. What about scenarios when I manually distribute the incoming relation using Iceberg functions? It still does not work right now as the other side is not `KeyGroupPartitioning`, despite having compatible keys.



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042670413


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   I'm mostly interested in
   UnknownDF joined to Iceberg Table 
       If UnknownDF has no partitioning info and IcebergTable does, Spark requests a repartition of UnknownDF via KeyGroupPartitioning transforms, then does the SPJ



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041616643


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {

Review Comment:
   Will add tests for applicable DELETE, UPDATE, MERGE in this PR.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041611507


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =

Review Comment:
   One alternative name was `spark.sql.iceberg.split.preserve-partition-boundaries`. I discarded it because we are not really preserving partition boundaries if there are multiple specs. Also, Spark will push down join keys in the future so we won't really respect partition boundaries as such (arguable).
   
   That being said, I'll think more tomorrow. Ideas are always welcome.
   
   



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


[GitHub] [iceberg] aokolnychyi commented on pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#issuecomment-1340218596

   cc @sunchao @flyrain @szehon-ho @RussellSpitzer @rdblue 


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041626586


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java:
##########
@@ -404,15 +404,15 @@ public void testSparkTableAddDropPartitions() throws Exception {
     assertPartitioningEquals(sparkTable(), 1, "bucket(16, id)");
 
     sql("ALTER TABLE %s ADD PARTITION FIELD truncate(data, 4)", tableName);
-    assertPartitioningEquals(sparkTable(), 2, "truncate(data, 4)");
+    assertPartitioningEquals(sparkTable(), 2, "truncate(4, data)");

Review Comment:
   I had to change `Spark3Util.toTransforms` to match `TruncateFunction`, which expects width first.



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042415619


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   I see the example below



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042635649


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(

Review Comment:
   Chao is correct. I disable AQE (can cause one partition) and broadcasts (have higher priority than SPJ) to make sure SPJ kicks in. 



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042631790


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =
+      "spark.sql.iceberg.split.preserve-data-grouping";
+  public static final boolean PRESERVE_DATA_GROUPING_DEFAULT = false;

Review Comment:
   My worry is performance regressions that this may cause. There may be substantially more splits if this config is on. In order to benefit from SPJ, joins must have equality conditions on partition columns. Spark will propagate join conditions in the future. Our long-term plan may be to check if v2 bucketing enabled and whether we have join conditions on partition columns to return true by default. Even that will be suboptimal cause we don't know if SPJ would actually apply or we simply produced less dense splits.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053521508


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java:
##########
@@ -143,4 +148,32 @@ public synchronized void testDeleteWithConcurrentTableRefresh() throws Exception
     executorService.shutdown();
     Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES));
   }
+
+  @Test
+  public void testRuntimeFilteringWithReportedPartitioning() throws NoSuchTableException {
+    createAndInitPartitionedTable();
+
+    append(new Employee(1, "hr"), new Employee(3, "hr"));
+    append(new Employee(1, "hardware"), new Employee(2, "hardware"));
+
+    Map<String, String> sqlConf =
+        ImmutableMap.of(
+            SQLConf.V2_BUCKETING_ENABLED().key(),
+            "true",
+            SparkSQLProperties.PRESERVE_DATA_GROUPING,
+            "true");
+
+    withSQLConf(sqlConf, () -> sql("DELETE FROM %s WHERE id = 2", tableName));
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots()));
+
+    Snapshot currentSnapshot = table.currentSnapshot();
+    validateCopyOnWrite(currentSnapshot, "1", "1", "1");

Review Comment:
   Nit: it would be nice to not pass the counts as strings...
   
   Not that we should fix it in this PR. But maybe we can introduce a version of `validateCopyOnWrite` that accepts ints and calls the string version.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053556414


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.extensions;
+
+import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE;
+import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ;
+
+import java.util.Map;
+import org.apache.iceberg.RowLevelOperationMode;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.spark.sql.execution.SparkPlan;
+import org.apache.spark.sql.internal.SQLConf;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+public class TestStoragePartitionedJoinsInRowLevelOperations extends SparkExtensionsTestBase {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final Map<String, String> COMMON_TABLE_PROPERTIES =
+      ImmutableMap.of(
+          TableProperties.FORMAT_VERSION,
+          "2",
+          TableProperties.SPLIT_SIZE,
+          "16777216",
+          TableProperties.SPLIT_OPEN_FILE_COST,
+          "16777216");
+
+  // only v2 bucketing and preserve data grouping properties have to be enabled to trigger SPJ
+  // other properties are only to simplify testing and validation
+  private static final Map<String, String> ENABLED_SPJ_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {
+        SparkCatalogConfig.HIVE.catalogName(),
+        SparkCatalogConfig.HIVE.implementation(),
+        SparkCatalogConfig.HIVE.properties()
+      }
+    };
+  }
+
+  public TestStoragePartitionedJoinsInRowLevelOperations(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  @Test
+  public void testCopyOnWriteDeleteWithoutShuffles() {
+    checkDelete(COPY_ON_WRITE);
+  }
+
+  @Test
+  public void testMergeOnReadDeleteWithoutShuffles() {
+    checkDelete(MERGE_ON_READ);
+  }
+
+  private void checkDelete(RowLevelOperationMode mode) {
+    String createTableStmt =
+        "CREATE TABLE %s (id INT, salary INT, dep STRING)"
+            + "USING iceberg "
+            + "PARTITIONED BY (dep) "
+            + "TBLPROPERTIES (%s)";
+
+    sql(createTableStmt, tableName, tablePropsAsString(COMMON_TABLE_PROPERTIES));
+
+    append(tableName, "{ \"id\": 1, \"salary\": 100, \"dep\": \"hr\" }");
+    append(tableName, "{ \"id\": 2, \"salary\": 200, \"dep\": \"hr\" }");
+    append(tableName, "{ \"id\": 3, \"salary\": 300, \"dep\": \"hr\" }");
+    append(tableName, "{ \"id\": 4, \"salary\": 400, \"dep\": \"hardware\" }");
+
+    sql(createTableStmt, tableName(OTHER_TABLE_NAME), tablePropsAsString(COMMON_TABLE_PROPERTIES));
+
+    append(tableName(OTHER_TABLE_NAME), "{ \"id\": 1, \"salary\": 110, \"dep\": \"hr\" }");
+    append(tableName(OTHER_TABLE_NAME), "{ \"id\": 5, \"salary\": 500, \"dep\": \"hr\" }");
+
+    Map<String, String> deleteTableProps =
+        ImmutableMap.of(
+            TableProperties.DELETE_MODE,
+            mode.modeName(),
+            TableProperties.DELETE_DISTRIBUTION_MODE,
+            "none");
+
+    sql("ALTER TABLE %s SET TBLPROPERTIES(%s)", tableName, tablePropsAsString(deleteTableProps));
+
+    withSQLConf(
+        ENABLED_SPJ_SQL_CONF,
+        () -> {
+          SparkPlan plan =
+              executeAndKeepPlan(
+                  "DELETE FROM %s t WHERE "
+                      + "EXISTS (SELECT 1 FROM %s s WHERE t.id = s.id AND t.dep = s.dep) AND "
+                      + "dep = 'hr'",

Review Comment:
   Why restrict this to just `hr`? It seems like that could cause everything to fit in a single task, which Spark handles specially in some cases. If we were to test a delete in both departments, then I think this would be more likely to test what we want in Spark.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055953155


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType = computeGroupingKeyType();
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private StructType computeGroupingKeyType() {
+    return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    if (groupingKeyTransforms == null) {
+      List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+      Set<Integer> seenFieldIds = Sets.newHashSet();
+
+      for (PartitionSpec spec : specs()) {
+        for (PartitionField field : spec.fields()) {
+          int fieldId = field.fieldId();
+
+          if (groupingKeyType().containsField(fieldId) && !seenFieldIds.contains(fieldId)) {
+            groupingKeyFields.add(field);
+          }
+
+          seenFieldIds.add(fieldId);
+        }
+      }
+
+      this.groupingKeyTransforms = Spark3Util.toTransforms(table().schema(), groupingKeyFields);
+    }
+
+    return groupingKeyTransforms;
+  }
+
+  protected Set<PartitionSpec> specs() {
+    if (specs == null) {
+      Set<PartitionSpec> taskSpecs = Sets.newHashSet();

Review Comment:
   `PartitionSpec.equals` can be expensive. Probably better to collect `spec.specId()` and then get the final set at the end.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1056852903


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java:
##########
@@ -140,4 +144,35 @@ public synchronized void testUpdateWithConcurrentTableRefresh() throws Exception
     executorService.shutdown();
     Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES));
   }
+
+  @Test
+  public void testRuntimeFilteringWithReportedPartitioning() {
+    createAndInitTable("id INT, dep STRING");
+    sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName);
+
+    append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 3, \"dep\": \"hr\" }");
+    append(
+        tableName,
+        "{ \"id\": 1, \"dep\": \"hardware\" }\n" + "{ \"id\": 2, \"dep\": \"hardware\" }");
+
+    Map<String, String> sqlConf =
+        ImmutableMap.of(
+            SQLConf.V2_BUCKETING_ENABLED().key(),
+            "true",
+            SparkSQLProperties.PRESERVE_DATA_GROUPING,
+            "true");
+
+    withSQLConf(sqlConf, () -> sql("UPDATE %s SET id = cast('-1' AS INT) WHERE id = 2", tableName));

Review Comment:
   Removed.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1056862319


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType = computeGroupingKeyType();
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private StructType computeGroupingKeyType() {
+    return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    if (groupingKeyTransforms == null) {
+      List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+      Set<Integer> seenFieldIds = Sets.newHashSet();
+
+      for (PartitionSpec spec : specs()) {
+        for (PartitionField field : spec.fields()) {
+          int fieldId = field.fieldId();
+
+          if (groupingKeyType().containsField(fieldId) && !seenFieldIds.contains(fieldId)) {
+            groupingKeyFields.add(field);
+          }
+
+          seenFieldIds.add(fieldId);
+        }
+      }
+
+      this.groupingKeyTransforms = Spark3Util.toTransforms(table().schema(), groupingKeyFields);
+    }
+
+    return groupingKeyTransforms;
+  }
+
+  protected Set<PartitionSpec> specs() {
+    if (specs == null) {
+      Set<PartitionSpec> taskSpecs = Sets.newHashSet();
+      for (T task : tasks()) {
+        taskSpecs.add(task.spec());
+      }
+      this.specs = taskSpecs;
+    }
+
+    return specs;
+  }
+
+  protected synchronized List<T> tasks() {
+    if (tasks == null) {
+      try (CloseableIterable<? extends ScanTask> taskIterable = scan.planFiles()) {
+        List<T> plannedTasks = Lists.newArrayList();
+
+        for (ScanTask task : taskIterable) {
+          ValidationException.check(
+              taskJavaClass().isInstance(task),
+              "Unsupported task type, expected a subtype of %s: %",
+              taskJavaClass().getName(),
+              task.getClass().getName());
+
+          plannedTasks.add(taskJavaClass().cast(task));
+        }
+
+        LOG.debug("Planned {} tasks", plannedTasks.size());
+
+        this.tasks = plannedTasks;
+      } catch (IOException e) {
+        throw new UncheckedIOException("Failed to close scan: " + scan, e);
+      }
+    }
+
+    return tasks;
+  }
+
+  @Override
+  protected synchronized List<ScanTaskGroup<T>> taskGroups() {
+    if (taskGroups == null) {
+      if (groupingKeyType().fields().isEmpty()) {
+        CloseableIterable<ScanTaskGroup<T>> plannedTaskGroups =
+            TableScanUtil.planTaskGroups(
+                CloseableIterable.withNoopClose(tasks()),
+                scan.targetSplitSize(),
+                scan.splitLookback(),
+                scan.splitOpenFileCost());
+        this.taskGroups = Lists.newArrayList(plannedTaskGroups);
+
+        LOG.debug("Planned {} task group(s) without data grouping", taskGroups.size());
+
+      } else {
+        List<ScanTaskGroup<T>> plannedTaskGroups =
+            TableScanUtil.planTaskGroups(
+                tasks(),
+                scan.targetSplitSize(),
+                scan.splitLookback(),
+                scan.splitOpenFileCost(),
+                groupingKeyType());
+        StructLikeSet plannedGroupingKeys = groupingKeys(plannedTaskGroups);
+
+        LOG.debug(

Review Comment:
   It may be helpful to know how many splits we had before and after runtime filtering.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053566628


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.extensions;
+
+import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE;
+import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ;
+
+import java.util.Map;
+import org.apache.iceberg.RowLevelOperationMode;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.spark.sql.execution.SparkPlan;
+import org.apache.spark.sql.internal.SQLConf;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+public class TestStoragePartitionedJoinsInRowLevelOperations extends SparkExtensionsTestBase {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final Map<String, String> COMMON_TABLE_PROPERTIES =
+      ImmutableMap.of(
+          TableProperties.FORMAT_VERSION,
+          "2",
+          TableProperties.SPLIT_SIZE,
+          "16777216",
+          TableProperties.SPLIT_OPEN_FILE_COST,
+          "16777216");
+
+  // only v2 bucketing and preserve data grouping properties have to be enabled to trigger SPJ
+  // other properties are only to simplify testing and validation
+  private static final Map<String, String> ENABLED_SPJ_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {
+        SparkCatalogConfig.HIVE.catalogName(),
+        SparkCatalogConfig.HIVE.implementation(),
+        SparkCatalogConfig.HIVE.properties()
+      }
+    };
+  }
+
+  public TestStoragePartitionedJoinsInRowLevelOperations(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  @Test
+  public void testCopyOnWriteDeleteWithoutShuffles() {
+    checkDelete(COPY_ON_WRITE);
+  }
+
+  @Test
+  public void testMergeOnReadDeleteWithoutShuffles() {
+    checkDelete(MERGE_ON_READ);
+  }
+
+  private void checkDelete(RowLevelOperationMode mode) {
+    String createTableStmt =
+        "CREATE TABLE %s (id INT, salary INT, dep STRING)"
+            + "USING iceberg "
+            + "PARTITIONED BY (dep) "
+            + "TBLPROPERTIES (%s)";
+
+    sql(createTableStmt, tableName, tablePropsAsString(COMMON_TABLE_PROPERTIES));
+
+    append(tableName, "{ \"id\": 1, \"salary\": 100, \"dep\": \"hr\" }");
+    append(tableName, "{ \"id\": 2, \"salary\": 200, \"dep\": \"hr\" }");
+    append(tableName, "{ \"id\": 3, \"salary\": 300, \"dep\": \"hr\" }");
+    append(tableName, "{ \"id\": 4, \"salary\": 400, \"dep\": \"hardware\" }");
+
+    sql(createTableStmt, tableName(OTHER_TABLE_NAME), tablePropsAsString(COMMON_TABLE_PROPERTIES));
+
+    append(tableName(OTHER_TABLE_NAME), "{ \"id\": 1, \"salary\": 110, \"dep\": \"hr\" }");
+    append(tableName(OTHER_TABLE_NAME), "{ \"id\": 5, \"salary\": 500, \"dep\": \"hr\" }");
+
+    Map<String, String> deleteTableProps =
+        ImmutableMap.of(
+            TableProperties.DELETE_MODE,
+            mode.modeName(),
+            TableProperties.DELETE_DISTRIBUTION_MODE,
+            "none");
+
+    sql("ALTER TABLE %s SET TBLPROPERTIES(%s)", tableName, tablePropsAsString(deleteTableProps));
+
+    withSQLConf(
+        ENABLED_SPJ_SQL_CONF,
+        () -> {
+          SparkPlan plan =
+              executeAndKeepPlan(
+                  "DELETE FROM %s t WHERE "
+                      + "EXISTS (SELECT 1 FROM %s s WHERE t.id = s.id AND t.dep = s.dep) AND "
+                      + "dep = 'hr'",

Review Comment:
   SPJ requires the same set of keys on both sides. The delta relation contains only records in `hr`. If I skip this, SPJ would not apply as the target has both `hr` and `hardware`. I make sure we don't have a single partition by configuring the open file cost and writing more than 1 file.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053566628


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.extensions;
+
+import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE;
+import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ;
+
+import java.util.Map;
+import org.apache.iceberg.RowLevelOperationMode;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.spark.sql.execution.SparkPlan;
+import org.apache.spark.sql.internal.SQLConf;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+public class TestStoragePartitionedJoinsInRowLevelOperations extends SparkExtensionsTestBase {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final Map<String, String> COMMON_TABLE_PROPERTIES =
+      ImmutableMap.of(
+          TableProperties.FORMAT_VERSION,
+          "2",
+          TableProperties.SPLIT_SIZE,
+          "16777216",
+          TableProperties.SPLIT_OPEN_FILE_COST,
+          "16777216");
+
+  // only v2 bucketing and preserve data grouping properties have to be enabled to trigger SPJ
+  // other properties are only to simplify testing and validation
+  private static final Map<String, String> ENABLED_SPJ_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {
+        SparkCatalogConfig.HIVE.catalogName(),
+        SparkCatalogConfig.HIVE.implementation(),
+        SparkCatalogConfig.HIVE.properties()
+      }
+    };
+  }
+
+  public TestStoragePartitionedJoinsInRowLevelOperations(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  @Test
+  public void testCopyOnWriteDeleteWithoutShuffles() {
+    checkDelete(COPY_ON_WRITE);
+  }
+
+  @Test
+  public void testMergeOnReadDeleteWithoutShuffles() {
+    checkDelete(MERGE_ON_READ);
+  }
+
+  private void checkDelete(RowLevelOperationMode mode) {
+    String createTableStmt =
+        "CREATE TABLE %s (id INT, salary INT, dep STRING)"
+            + "USING iceberg "
+            + "PARTITIONED BY (dep) "
+            + "TBLPROPERTIES (%s)";
+
+    sql(createTableStmt, tableName, tablePropsAsString(COMMON_TABLE_PROPERTIES));
+
+    append(tableName, "{ \"id\": 1, \"salary\": 100, \"dep\": \"hr\" }");
+    append(tableName, "{ \"id\": 2, \"salary\": 200, \"dep\": \"hr\" }");
+    append(tableName, "{ \"id\": 3, \"salary\": 300, \"dep\": \"hr\" }");
+    append(tableName, "{ \"id\": 4, \"salary\": 400, \"dep\": \"hardware\" }");
+
+    sql(createTableStmt, tableName(OTHER_TABLE_NAME), tablePropsAsString(COMMON_TABLE_PROPERTIES));
+
+    append(tableName(OTHER_TABLE_NAME), "{ \"id\": 1, \"salary\": 110, \"dep\": \"hr\" }");
+    append(tableName(OTHER_TABLE_NAME), "{ \"id\": 5, \"salary\": 500, \"dep\": \"hr\" }");
+
+    Map<String, String> deleteTableProps =
+        ImmutableMap.of(
+            TableProperties.DELETE_MODE,
+            mode.modeName(),
+            TableProperties.DELETE_DISTRIBUTION_MODE,
+            "none");
+
+    sql("ALTER TABLE %s SET TBLPROPERTIES(%s)", tableName, tablePropsAsString(deleteTableProps));
+
+    withSQLConf(
+        ENABLED_SPJ_SQL_CONF,
+        () -> {
+          SparkPlan plan =
+              executeAndKeepPlan(
+                  "DELETE FROM %s t WHERE "
+                      + "EXISTS (SELECT 1 FROM %s s WHERE t.id = s.id AND t.dep = s.dep) AND "
+                      + "dep = 'hr'",

Review Comment:
   SPJ requires the same set of keys on both sides. The delta relation contains only records in `hr`. If I skip this, SPJ would not apply as the target has both `hr` and `hardware`. I make sure we don't have a single partition by configuring the split size and writing more than 1 file.



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042433535


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   Does this work with a non Iceberg Source as part of the join?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042710634


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   I'd be great to at least support cases when users manually repartition the incoming source relation using the Iceberg function catalog. I believe that would produce `HashPartitioning` or `RangePartitioning` on the other side.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042622671


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =

Review Comment:
   I am afraid this one will be pretty public. Users will have to explicitly enable this as we don't know if Spark can benefit from the reported distribution and skip shuffles.



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


[GitHub] [iceberg] sunchao commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042659356


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   In this case, is the non-Iceberg side partitioned? If so, there could be an issue if we want to shuffle this side using partition transforms reported on the Iceberg side, since we don't know the value space for the non-partitioned side. For instance, the partitioned Iceberg table could contain partitions `[0, 1]` but the non-partitioned non-Iceberg side could have `[0, 1, 2]`. It could work only for `bucket`.



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


[GitHub] [iceberg] sunchao commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
sunchao commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1042685029


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {
+
+  private static final String OTHER_TABLE_NAME = "other_table";
+
+  // open file cost and split size are set as 16 MB to produce a split per file
+  private static final String TABLE_PROPERTIES =
+      String.format(
+          "'%s' = 16777216, '%s' = 16777216",
+          TableProperties.SPLIT_SIZE, TableProperties.SPLIT_OPEN_FILE_COST);
+
+  private static final Map<String, String> SPJ_ON_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "true",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  private static final Map<String, String> SPJ_OFF_SQL_CONF =
+      ImmutableMap.of(
+          SQLConf.V2_BUCKETING_ENABLED().key(),
+          "false",
+          SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(),
+          "false",
+          SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(),
+          "false",
+          SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(),
+          "-1",
+          SparkSQLProperties.PRESERVE_DATA_GROUPING,
+          "true");
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+    sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME));
+  }
+
+  // TODO: add tests for truncate transforms once SPARK-40295 is released
+  // TODO: add tests for cases when one side contains a subset of keys once Spark supports this
+

Review Comment:
   For inner-join, yes it is probably do-able, but it won't work for outer-joins.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041611507


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =

Review Comment:
   One alternative name was `spark.sql.iceberg.split.preserve-partition-boundaries`. I discarded it because we are not really preserving partition boundaries if there are multiple specs. Also, Spark will push down join keys in the future so we won't really respect partition boundaries as such.
   
   That being said, I'll think more tomorrow. Ideas are always welcome.
   
   



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041626586


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java:
##########
@@ -404,15 +404,15 @@ public void testSparkTableAddDropPartitions() throws Exception {
     assertPartitioningEquals(sparkTable(), 1, "bucket(16, id)");
 
     sql("ALTER TABLE %s ADD PARTITION FIELD truncate(data, 4)", tableName);
-    assertPartitioningEquals(sparkTable(), 2, "truncate(data, 4)");
+    assertPartitioningEquals(sparkTable(), 2, "truncate(4, data)");

Review Comment:
   I had to change `Spark3Util.toTransforms` to match `TruncateFunction`, which assumes the width first.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1041614262


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java:
##########
@@ -356,4 +357,23 @@ private <T> GenericArrayData fillArray(
 
     return new GenericArrayData(array);
   }
+
+  @Override
+  public boolean equals(Object other) {

Review Comment:
   Had to implement as Spark collects grouping keys and checks their equality.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055879720


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -42,4 +42,9 @@ private SparkSQLProperties() {}
   // Controls whether to check the order of fields during writes
   public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering";
   public static final boolean CHECK_ORDERING_DEFAULT = true;
+
+  // Controls whether to preserve the existing grouping of data while planning splits
+  public static final String PRESERVE_DATA_GROUPING =
+      "spark.sql.iceberg.split.preserve-data-grouping";

Review Comment:
   I was inspired by `read.split` table properties we have. If omitted, it is not clear whether it applies only to reads.
   I like `planning`. We could also go for `read.split` to follow table properties but either one is equally good for me.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053518496


##########
core/src/test/java/org/apache/iceberg/TestPartitioning.java:
##########
@@ -354,6 +354,19 @@ public void testGroupingKeyTypeWithEvolvedUnpartitionedSpec() {
     Assert.assertEquals("Types must match", expectedType, actualType);
   }
 
+  @Test
+  public void testGroupingKeyTypeWithProjectedSchema() {
+    TestTables.TestTable table =
+        TestTables.create(tableDir, "test", SCHEMA, BY_CATEGORY_DATA_SPEC, V1_FORMAT_VERSION);
+
+    Schema projectedSchema = table.schema().select("id", "data");

Review Comment:
   Ah, so is the idea to use the pushed projection to limit the fields used? I think that's a great idea!



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053517623


##########
api/src/main/java/org/apache/iceberg/types/Types.java:
##########
@@ -554,6 +554,10 @@ public List<NestedField> fields() {
       return lazyFieldList();
     }
 
+    public boolean containsField(int id) {

Review Comment:
   Is this helpful? Further down, you use `schema.findField(field.sourceId()) != null` to check whether a schema contains a field (at any level) and that seems like a reasonable check to me.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1052658901


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,585 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {

Review Comment:
   Done. Resolving.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1056852950


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType = computeGroupingKeyType();
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private StructType computeGroupingKeyType() {
+    return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    if (groupingKeyTransforms == null) {
+      List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+      Set<Integer> seenFieldIds = Sets.newHashSet();
+
+      for (PartitionSpec spec : specs()) {

Review Comment:
   Fixed.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType = computeGroupingKeyType();
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private StructType computeGroupingKeyType() {
+    return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    if (groupingKeyTransforms == null) {
+      List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+      Set<Integer> seenFieldIds = Sets.newHashSet();
+
+      for (PartitionSpec spec : specs()) {
+        for (PartitionField field : spec.fields()) {
+          int fieldId = field.fieldId();
+
+          if (groupingKeyType().containsField(fieldId) && !seenFieldIds.contains(fieldId)) {
+            groupingKeyFields.add(field);
+          }
+
+          seenFieldIds.add(fieldId);
+        }
+      }
+
+      this.groupingKeyTransforms = Spark3Util.toTransforms(table().schema(), groupingKeyFields);
+    }
+
+    return groupingKeyTransforms;
+  }
+
+  protected Set<PartitionSpec> specs() {
+    if (specs == null) {
+      Set<PartitionSpec> taskSpecs = Sets.newHashSet();

Review Comment:
   Fixed.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType = computeGroupingKeyType();
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private StructType computeGroupingKeyType() {
+    return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    if (groupingKeyTransforms == null) {
+      List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+      Set<Integer> seenFieldIds = Sets.newHashSet();
+
+      for (PartitionSpec spec : specs()) {
+        for (PartitionField field : spec.fields()) {
+          int fieldId = field.fieldId();
+
+          if (groupingKeyType().containsField(fieldId) && !seenFieldIds.contains(fieldId)) {
+            groupingKeyFields.add(field);
+          }
+
+          seenFieldIds.add(fieldId);
+        }
+      }
+
+      this.groupingKeyTransforms = Spark3Util.toTransforms(table().schema(), groupingKeyFields);
+    }
+
+    return groupingKeyTransforms;
+  }
+
+  protected Set<PartitionSpec> specs() {
+    if (specs == null) {
+      Set<PartitionSpec> taskSpecs = Sets.newHashSet();
+      for (T task : tasks()) {
+        taskSpecs.add(task.spec());
+      }
+      this.specs = taskSpecs;
+    }
+
+    return specs;
+  }
+
+  protected synchronized List<T> tasks() {
+    if (tasks == null) {
+      try (CloseableIterable<? extends ScanTask> taskIterable = scan.planFiles()) {
+        List<T> plannedTasks = Lists.newArrayList();
+
+        for (ScanTask task : taskIterable) {
+          ValidationException.check(
+              taskJavaClass().isInstance(task),
+              "Unsupported task type, expected a subtype of %s: %",
+              taskJavaClass().getName(),
+              task.getClass().getName());
+
+          plannedTasks.add(taskJavaClass().cast(task));
+        }
+
+        LOG.debug("Planned {} tasks", plannedTasks.size());

Review Comment:
   Removed.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1056852402


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java:
##########
@@ -356,4 +357,23 @@ private <T> GenericArrayData fillArray(
 
     return new GenericArrayData(array);
   }
+
+  @Override
+  public boolean equals(Object other) {

Review Comment:
   It is a bug in Spark that will be fixed. Spark uses `groupBy` in one place, which relies on equals.
   All other places use `InternalRowSet`.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1055953968


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.BaseScanTaskGroup;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Scan;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.ScanTaskGroup;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadConf;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class SparkPartitioningAwareScan<T extends PartitionScanTask> extends SparkScan
+    implements SupportsReportPartitioning {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class);
+
+  private final Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan;
+  private final boolean preserveDataGrouping;
+
+  private Set<PartitionSpec> specs = null; // lazy cache of scanned specs
+  private List<T> tasks = null; // lazy cache of uncombined tasks
+  private List<ScanTaskGroup<T>> taskGroups = null; // lazy cache of task groups
+  private StructType groupingKeyType = null; // lazy cache of the grouping key type
+  private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms
+  private StructLikeSet groupingKeys = null; // lazy cache of grouping keys
+
+  SparkPartitioningAwareScan(
+      SparkSession spark,
+      Table table,
+      Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan,
+      SparkReadConf readConf,
+      Schema expectedSchema,
+      List<Expression> filters) {
+
+    super(spark, table, readConf, expectedSchema, filters);
+
+    this.scan = scan;
+    this.preserveDataGrouping = readConf.preserveDataGrouping();
+
+    if (scan == null) {
+      this.specs = Collections.emptySet();
+      this.tasks = Collections.emptyList();
+      this.taskGroups = Collections.emptyList();
+    }
+  }
+
+  protected abstract Class<T> taskJavaClass();
+
+  protected Scan<?, ? extends ScanTask, ? extends ScanTaskGroup<?>> scan() {
+    return scan;
+  }
+
+  @Override
+  public Partitioning outputPartitioning() {
+    if (groupingKeyType().fields().isEmpty()) {
+      LOG.info("Reporting UnknownPartitioning with {} partition(s)", taskGroups().size());
+      return new UnknownPartitioning(taskGroups().size());
+    } else {
+      LOG.info(
+          "Reporting KeyGroupedPartitioning by {} with {} partition(s)",
+          groupingKeyTransforms(),
+          taskGroups().size());
+      return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size());
+    }
+  }
+
+  @Override
+  protected StructType groupingKeyType() {
+    if (groupingKeyType == null) {
+      if (preserveDataGrouping) {
+        this.groupingKeyType = computeGroupingKeyType();
+      } else {
+        this.groupingKeyType = StructType.of();
+      }
+    }
+
+    return groupingKeyType;
+  }
+
+  private StructType computeGroupingKeyType() {
+    return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs());
+  }
+
+  private Transform[] groupingKeyTransforms() {
+    if (groupingKeyTransforms == null) {
+      List<PartitionField> groupingKeyFields = Lists.newArrayList();
+
+      Set<Integer> seenFieldIds = Sets.newHashSet();
+
+      for (PartitionSpec spec : specs()) {
+        for (PartitionField field : spec.fields()) {
+          int fieldId = field.fieldId();
+
+          if (groupingKeyType().containsField(fieldId) && !seenFieldIds.contains(fieldId)) {
+            groupingKeyFields.add(field);
+          }
+
+          seenFieldIds.add(fieldId);
+        }
+      }
+
+      this.groupingKeyTransforms = Spark3Util.toTransforms(table().schema(), groupingKeyFields);
+    }
+
+    return groupingKeyTransforms;
+  }
+
+  protected Set<PartitionSpec> specs() {
+    if (specs == null) {
+      Set<PartitionSpec> taskSpecs = Sets.newHashSet();
+      for (T task : tasks()) {
+        taskSpecs.add(task.spec());
+      }
+      this.specs = taskSpecs;
+    }
+
+    return specs;
+  }
+
+  protected synchronized List<T> tasks() {
+    if (tasks == null) {
+      try (CloseableIterable<? extends ScanTask> taskIterable = scan.planFiles()) {
+        List<T> plannedTasks = Lists.newArrayList();
+
+        for (ScanTask task : taskIterable) {
+          ValidationException.check(
+              taskJavaClass().isInstance(task),
+              "Unsupported task type, expected a subtype of %s: %",
+              taskJavaClass().getName(),
+              task.getClass().getName());
+
+          plannedTasks.add(taskJavaClass().cast(task));
+        }
+
+        LOG.debug("Planned {} tasks", plannedTasks.size());

Review Comment:
   This information will be logged at a higher level with scan metrics, so no need for debug 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-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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053570254


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java:
##########
@@ -0,0 +1,586 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkSQLProperties;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.spark.data.RandomData;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestStoragePartitionedJoins extends SparkTestBaseWithCatalog {

Review Comment:
   @rdblue, this is the primary test suite that covers different transforms. I had just a few tests for row-level operations.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #6371: Spark 3.3: Support storage-partitioned joins

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #6371:
URL: https://github.com/apache/iceberg/pull/6371#discussion_r1053560460


##########
core/src/main/java/org/apache/iceberg/Partitioning.java:
##########
@@ -215,11 +225,15 @@ public Void alwaysNull(int fieldId, String sourceName, int sourceId) {
    * that have the same field ID but use a void transform under the hood. Such fields cannot be part
    * of the grouping key as void transforms always return null.
    *
+   * <p>If the provided schema is not null, this method will only take into account partition fields
+   * on top of columns present in the schema. Otherwise, all partition fields will be considered.
+   *
+   * @param schema a schema specifying a set of source columns to consider (null to consider all)
    * @param specs one or many specs
    * @return the constructed grouping key type
    */
-  public static StructType groupingKeyType(Collection<PartitionSpec> specs) {
-    return buildPartitionProjectionType("grouping key", specs, commonActiveFieldIds(specs));
+  public static StructType groupingKeyType(Schema schema, Collection<PartitionSpec> specs) {
+    return buildPartitionProjectionType("grouping key", specs, commonActiveFieldIds(schema, specs));

Review Comment:
   If a table has multiple specs but we scan only tasks that belong to one spec, we should take into account only the one that is being queried. 



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