You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "huaxingao (via GitHub)" <gi...@apache.org> on 2023/02/23 20:20:23 UTC

[GitHub] [iceberg] huaxingao opened a new pull request, #6923: fix `TestAggregatePushDown`

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

   - explicitly set the String column's mode to `truncate` for aggregate push down test for String
   - convert explain string to lower case in  aggregate push down tests
   


-- 
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] huaxingao commented on a diff in pull request #6923: Aggregate Push Down Followup

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#discussion_r1117806725


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java:
##########
@@ -32,10 +33,14 @@ class SparkLocalScan implements LocalScan {
   private final StructType readSchema;
   private final InternalRow[] rows;
 

Review Comment:
   removed



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java:
##########
@@ -50,8 +55,13 @@ public StructType readSchema() {
 
   @Override
   public String description() {
-    String fields =
-        Arrays.stream(readSchema.fields()).map(StructField::name).collect(Collectors.joining(", "));
-    return String.format("%s [%s]", table, fields);
+    return String.format("%s [filters=%s]", table, Spark3Util.describe(filterExpressions));
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "IcebergScan(table=%s, type=%s, filters=%s)",

Review Comment:
   Fixed. Thanks



-- 
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 pull request #6923: Aggregate Push Down Followup

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#issuecomment-1445450290

   Thanks, @huaxingao!


-- 
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] huaxingao commented on pull request #6923: Aggregate Push Down Followup

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#issuecomment-1445451695

   Thank you 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] huaxingao commented on a diff in pull request #6923: Aggregate Push Down Followup

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#discussion_r1117828901


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java:
##########
@@ -48,31 +48,20 @@ public static Expression convert(AggregateFunc aggregate) {
         case COUNT:
           Count countAgg = (Count) aggregate;
           if (countAgg.isDistinct()) {
-            // manifest file doesn't have count distinct so this can't be converted to push down
-            return null;
-          }
-
-          if (countAgg.column() instanceof NamedReference) {
-            return Expressions.count(SparkUtil.toColumnName((NamedReference) countAgg.column()));
-          } else {
             return null;
           }
+          assert (countAgg.column() instanceof NamedReference);
+          return Expressions.count(SparkUtil.toColumnName((NamedReference) countAgg.column()));

Review Comment:
   Changed to `if/else`



-- 
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] huaxingao commented on a diff in pull request #6923: Aggregate Push Down Followup

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#discussion_r1117806287


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkAggregates.java:
##########
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iceberg.spark.source;
-
-import java.util.Map;
-import org.apache.iceberg.expressions.Expression;
-import org.apache.iceberg.expressions.Expression.Operation;
-import org.apache.iceberg.expressions.Expressions;
-import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
-import org.apache.iceberg.spark.SparkUtil;
-import org.apache.spark.sql.connector.expressions.NamedReference;
-import org.apache.spark.sql.connector.expressions.aggregate.AggregateFunc;
-import org.apache.spark.sql.connector.expressions.aggregate.Count;
-import org.apache.spark.sql.connector.expressions.aggregate.CountStar;
-import org.apache.spark.sql.connector.expressions.aggregate.Max;
-import org.apache.spark.sql.connector.expressions.aggregate.Min;
-
-public class SparkAggregates {
-
-  private SparkAggregates() {}
-
-  private static final Map<Class<? extends AggregateFunc>, Operation> AGGREGATES =
-      ImmutableMap.<Class<? extends AggregateFunc>, Operation>builder()
-          .put(Count.class, Operation.COUNT)
-          .put(CountStar.class, Operation.COUNT_STAR)
-          .put(Max.class, Operation.MAX)
-          .put(Min.class, Operation.MIN)
-          .build();
-
-  public static Expression convert(AggregateFunc aggregate) {
-    Operation op = AGGREGATES.get(aggregate.getClass());
-    if (op != null) {
-      switch (op) {
-        case COUNT:
-          Count countAgg = (Count) aggregate;
-          assert (countAgg.column() instanceof NamedReference);

Review Comment:
   We still need the `isDistinct()`, because we don't want to push down `count(distinct c)`



-- 
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 #6923: Aggregate Push Down Followup

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#discussion_r1117780025


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -50,7 +50,7 @@ private SparkSQLProperties() {}
 
   // Controls whether to push down aggregate (MAX/MIN/COUNT) to Iceberg
   public static final String AGGREGATE_PUSH_DOWN_ENABLED =
-      "spark.sql.iceberg.aggregate-push-down-enabled";
+      "spark.sql.iceberg.aggregate-push-down.enabled";

Review Comment:
   @rdblue, I asked for this rename. Let me know what you think. My reasoning is [here](https://github.com/apache/iceberg/pull/6622/files#r1117666975).



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java:
##########
@@ -50,8 +55,13 @@ public StructType readSchema() {
 
   @Override
   public String description() {
-    String fields =
-        Arrays.stream(readSchema.fields()).map(StructField::name).collect(Collectors.joining(", "));
-    return String.format("%s [%s]", table, fields);
+    return String.format("%s [filters=%s]", table, Spark3Util.describe(filterExpressions));
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "IcebergScan(table=%s, type=%s, filters=%s)",

Review Comment:
   What about `IcebergScan` -> `IcebergLocalScan` to indicate it is a local scan?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java:
##########
@@ -32,10 +33,14 @@ class SparkLocalScan implements LocalScan {
   private final StructType readSchema;
   private final InternalRow[] rows;
 

Review Comment:
   nit: Shall we group all vars together? There is an empty line before filters now.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkAggregates.java:
##########
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.iceberg.spark.source;
-
-import java.util.Map;
-import org.apache.iceberg.expressions.Expression;
-import org.apache.iceberg.expressions.Expression.Operation;
-import org.apache.iceberg.expressions.Expressions;
-import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
-import org.apache.iceberg.spark.SparkUtil;
-import org.apache.spark.sql.connector.expressions.NamedReference;
-import org.apache.spark.sql.connector.expressions.aggregate.AggregateFunc;
-import org.apache.spark.sql.connector.expressions.aggregate.Count;
-import org.apache.spark.sql.connector.expressions.aggregate.CountStar;
-import org.apache.spark.sql.connector.expressions.aggregate.Max;
-import org.apache.spark.sql.connector.expressions.aggregate.Min;
-
-public class SparkAggregates {
-
-  private SparkAggregates() {}
-
-  private static final Map<Class<? extends AggregateFunc>, Operation> AGGREGATES =
-      ImmutableMap.<Class<? extends AggregateFunc>, Operation>builder()
-          .put(Count.class, Operation.COUNT)
-          .put(CountStar.class, Operation.COUNT_STAR)
-          .put(Max.class, Operation.MAX)
-          .put(Min.class, Operation.MIN)
-          .build();
-
-  public static Expression convert(AggregateFunc aggregate) {
-    Operation op = AGGREGATES.get(aggregate.getClass());
-    if (op != null) {
-      switch (op) {
-        case COUNT:
-          Count countAgg = (Count) aggregate;
-          assert (countAgg.column() instanceof NamedReference);

Review Comment:
   I kind of like not having if/else blocks compared to the remaining `SparkAggregates` utility we kept. Shall we adapt the one we kept to match this one? We probably just need to return null instead of throwing an exception.
   
   Also, I see the kept utility has a special condition for `isDistinct()`. Is that still needed? We don't have it here.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java:
##########
@@ -188,13 +189,12 @@ public boolean pushAggregation(Aggregation aggregation) {
         if (expr != null) {
           Expression bound = Binder.bind(schema.asStruct(), expr, caseSensitive);
           expressions.add((BoundAggregate<?, ?>) bound);
+        } else {
+          LOG.info(
+              "Skipping aggregate pushdown: AggregateFunc {} can't be converted to iceberg Expression",

Review Comment:
   nit: `to iceberg Expression` -> `to Iceberg expression` or simply `to Iceberg`.



-- 
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 merged pull request #6923: Aggregate Push Down Followup

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue merged PR #6923:
URL: https://github.com/apache/iceberg/pull/6923


-- 
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 #6923: Aggregate Push Down Followup

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#discussion_r1117825305


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java:
##########
@@ -48,31 +48,20 @@ public static Expression convert(AggregateFunc aggregate) {
         case COUNT:
           Count countAgg = (Count) aggregate;
           if (countAgg.isDistinct()) {
-            // manifest file doesn't have count distinct so this can't be converted to push down
-            return null;
-          }
-
-          if (countAgg.column() instanceof NamedReference) {
-            return Expressions.count(SparkUtil.toColumnName((NamedReference) countAgg.column()));
-          } else {
             return null;
           }
+          assert (countAgg.column() instanceof NamedReference);
+          return Expressions.count(SparkUtil.toColumnName((NamedReference) countAgg.column()));

Review Comment:
   Oh, I see that this was from the other `SparkAggregates`. I think we should use the `if`/`else` 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] aokolnychyi commented on pull request #6923: Aggregate Push Down Followup

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#issuecomment-1444741508

   Thanks, @huaxingao!


-- 
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] huaxingao commented on a diff in pull request #6923: Aggregate Push Down Followup

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#discussion_r1117806859


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java:
##########
@@ -188,13 +189,12 @@ public boolean pushAggregation(Aggregation aggregation) {
         if (expr != null) {
           Expression bound = Binder.bind(schema.asStruct(), expr, caseSensitive);
           expressions.add((BoundAggregate<?, ?>) bound);
+        } else {
+          LOG.info(
+              "Skipping aggregate pushdown: AggregateFunc {} can't be converted to iceberg Expression",

Review Comment:
   Fixed



-- 
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] huaxingao commented on pull request #6923: Aggregate Push Down Followup

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#issuecomment-1444638936

   cc @aokolnychyi @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] rdblue commented on a diff in pull request #6923: Aggregate Push Down Followup

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#discussion_r1117825003


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java:
##########
@@ -48,31 +48,20 @@ public static Expression convert(AggregateFunc aggregate) {
         case COUNT:
           Count countAgg = (Count) aggregate;
           if (countAgg.isDistinct()) {
-            // manifest file doesn't have count distinct so this can't be converted to push down
-            return null;
-          }
-
-          if (countAgg.column() instanceof NamedReference) {
-            return Expressions.count(SparkUtil.toColumnName((NamedReference) countAgg.column()));
-          } else {
             return null;
           }
+          assert (countAgg.column() instanceof NamedReference);
+          return Expressions.count(SparkUtil.toColumnName((NamedReference) countAgg.column()));

Review Comment:
   I think this was actually better before. What was the rationale for converting to use `assert`? We don't use that elsewhere and I think the correct behavior is to return `null` when something is unexpected and can't be converted.



-- 
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 #6923: Aggregate Push Down Followup

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#discussion_r1117824744


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java:
##########
@@ -48,31 +48,20 @@ public static Expression convert(AggregateFunc aggregate) {
         case COUNT:
           Count countAgg = (Count) aggregate;
           if (countAgg.isDistinct()) {
-            // manifest file doesn't have count distinct so this can't be converted to push down
-            return null;
-          }
-
-          if (countAgg.column() instanceof NamedReference) {
-            return Expressions.count(SparkUtil.toColumnName((NamedReference) countAgg.column()));
-          } else {
             return null;
           }
+          assert (countAgg.column() instanceof NamedReference);

Review Comment:
   We don't usually include assertions. Can you update this to an `if` check that causes the method to return null?



-- 
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 #6923: Aggregate Push Down Followup

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #6923:
URL: https://github.com/apache/iceberg/pull/6923#discussion_r1117824258


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java:
##########
@@ -50,7 +50,7 @@ private SparkSQLProperties() {}
 
   // Controls whether to push down aggregate (MAX/MIN/COUNT) to Iceberg
   public static final String AGGREGATE_PUSH_DOWN_ENABLED =
-      "spark.sql.iceberg.aggregate-push-down-enabled";
+      "spark.sql.iceberg.aggregate-push-down.enabled";

Review Comment:
   Sounds good 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