-
Notifications
You must be signed in to change notification settings - Fork 3k
Spark: Fix IllegarlArgumentException when filtering on BinaryType column #3460
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,7 +19,6 @@ | |
|
|
||
| package org.apache.iceberg.spark; | ||
|
|
||
| import java.nio.ByteBuffer; | ||
| import java.util.Arrays; | ||
| import java.util.Collections; | ||
| import java.util.HashMap; | ||
|
|
@@ -636,17 +635,17 @@ public <T> String predicate(UnboundPredicate<T> pred) { | |
| case NOT_NAN: | ||
| return "not_nan(" + pred.ref().name() + ")"; | ||
| case LT: | ||
| return pred.ref().name() + " < " + sqlString(pred.literal()); | ||
| return pred.ref().name() + " < " + pred.literal(); | ||
| case LT_EQ: | ||
| return pred.ref().name() + " <= " + sqlString(pred.literal()); | ||
| return pred.ref().name() + " <= " + pred.literal(); | ||
| case GT: | ||
| return pred.ref().name() + " > " + sqlString(pred.literal()); | ||
| return pred.ref().name() + " > " + pred.literal(); | ||
| case GT_EQ: | ||
| return pred.ref().name() + " >= " + sqlString(pred.literal()); | ||
| return pred.ref().name() + " >= " + pred.literal(); | ||
| case EQ: | ||
| return pred.ref().name() + " = " + sqlString(pred.literal()); | ||
| return pred.ref().name() + " = " + pred.literal(); | ||
| case NOT_EQ: | ||
| return pred.ref().name() + " != " + sqlString(pred.literal()); | ||
| return pred.ref().name() + " != " + pred.literal(); | ||
| case STARTS_WITH: | ||
| return pred.ref().name() + " LIKE '" + pred.literal() + "%'"; | ||
| case IN: | ||
|
|
@@ -659,17 +658,7 @@ public <T> String predicate(UnboundPredicate<T> pred) { | |
| } | ||
|
|
||
| private static <T> String sqlString(List<org.apache.iceberg.expressions.Literal<T>> literals) { | ||
| return literals.stream().map(DescribeExpressionVisitor::sqlString).collect(Collectors.joining(", ")); | ||
| } | ||
|
|
||
| private static String sqlString(org.apache.iceberg.expressions.Literal<?> lit) { | ||
| if (lit.value() instanceof String) { | ||
| return "'" + lit.value() + "'"; | ||
| } else if (lit.value() instanceof ByteBuffer) { | ||
| throw new IllegalArgumentException("Cannot convert bytes to SQL literal: " + lit); | ||
| } else { | ||
| return lit.value().toString(); | ||
| } | ||
| return literals.stream().map(Object::toString).collect(Collectors.joining(", ")); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This seems possibly incorrect or a deviation from the behavior before. It looks like this code is calling Object.toString on
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks for your review. After testing, I think this is correct |
||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,7 @@ | |
|
|
||
| package org.apache.iceberg.spark; | ||
|
|
||
| import java.lang.reflect.Array; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.stream.Collectors; | ||
|
|
@@ -118,6 +119,8 @@ private Object[] toJava(Row row) { | |
| return row.getList(pos); | ||
| } else if (value instanceof scala.collection.Map) { | ||
| return row.getJavaMap(pos); | ||
| } else if (value.getClass().isArray() && value.getClass().getComponentType().isPrimitive()) { | ||
| return IntStream.range(0, Array.getLength(value)).mapToObj(i -> Array.get(value, i)).toArray(); | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think this is correct. The |
||
| } else { | ||
| return value; | ||
| } | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why did we get rid of the
sqlStringfunction here? Is there a change that removes the need to quote strings for example?Also, since the representation of
ByteBufferis potentially engine specific, would it be better to add the conversion to thesqlStringhere for Spark (e.g. using the leading capital X, likeX'123456')?This would allow other engines to handle it themselves (e.g. if Flink doesn't use leading
Xformat).There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@kbendick is right. This should not remove the
sqlStringfunction. That converts a literal into a form that is usable by Spark. Instead, you should update that function to produce Spark's binary literal format.Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
sqlStringis implemented and used in the DescribeExpressionVisitor class.I think this class is just for generating description information, and the description will be displayed in the log and Spark UI for debugging. So in my opinion, it is an acceptable way to use
Literal.toStringdirectly, which ensures that the description is the same as the style of other logs.In fact, except for the difference between StringLiteral's single and double quotes, the current
sqlStringandLiteral.toStringare the same.But if you think that keeping sqlString is a better way, I can also modify the code.
In addition, there are some differences between the description form of spark here, for example, the StringLiteral of spark does not have quotation marks. If you think that this should be the same as the behavior of spark, I can also modify the code.