-
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 3 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 |
|---|---|---|
|
|
@@ -35,6 +35,7 @@ | |
| import java.util.Objects; | ||
| import java.util.UUID; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
| import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; | ||
| import org.apache.iceberg.types.Comparators; | ||
| import org.apache.iceberg.types.Conversions; | ||
| import org.apache.iceberg.types.Type; | ||
|
|
@@ -599,6 +600,13 @@ protected Type.TypeID typeId() { | |
| Object writeReplace() throws ObjectStreamException { | ||
| return new SerializationProxies.FixedLiteralProxy(value()); | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| byte[] binary = new byte[value().remaining()]; | ||
| value().duplicate().get(binary); | ||
| return "0x" + BaseEncoding.base16().encode(binary); | ||
| } | ||
| } | ||
|
|
||
| static class BinaryLiteral extends BaseLiteral<ByteBuffer> { | ||
|
|
@@ -639,5 +647,12 @@ Object writeReplace() throws ObjectStreamException { | |
| protected Type.TypeID typeId() { | ||
| return Type.TypeID.BINARY; | ||
| } | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| byte[] binary = new byte[value().remaining()]; | ||
| value().duplicate().get(binary); | ||
|
||
| return "0x" + BaseEncoding.base16().encode(binary); | ||
|
||
| } | ||
| } | ||
| } | ||
| 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(); | ||
|
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. Why did we get rid of the Also, since the representation of This would allow other engines to handle it themselves (e.g. if Flink doesn't use leading
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. @kbendick is right. This should not remove the
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 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 In fact, except for the difference between StringLiteral's single and double quotes, the current But if you think that keeping sqlString is a better way, I can also modify the code. |
||
| 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 |
|---|---|---|
|
|
@@ -39,6 +39,7 @@ | |
| import org.junit.AfterClass; | ||
| import org.junit.Assert; | ||
| import org.junit.BeforeClass; | ||
| import org.junit.internal.ExactComparisonCriteria; | ||
|
||
|
|
||
| import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; | ||
|
|
||
|
|
@@ -157,7 +158,11 @@ private void assertEquals(String context, Object[] expectedRow, Object[] actualR | |
| Object actualValue = actualRow[col]; | ||
| if (expectedValue != null && expectedValue.getClass().isArray()) { | ||
| String newContext = String.format("%s (nested col %d)", context, col + 1); | ||
| assertEquals(newContext, (Object[]) expectedValue, (Object[]) actualValue); | ||
| if (expectedValue.getClass().getComponentType().isPrimitive()) { | ||
| new ExactComparisonCriteria().arrayEquals(newContext, expectedValue, actualValue); | ||
| } else { | ||
| assertEquals(newContext, (Object[]) expectedValue, (Object[]) actualValue); | ||
| } | ||
|
||
| } else if (expectedValue != ANY) { | ||
| Assert.assertEquals(context + " contents should match", expectedValue, actualValue); | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -49,8 +49,8 @@ public TestSelect(String catalogName, String implementation, Map<String, String> | |
|
|
||
| @Before | ||
| public void createTables() { | ||
| sql("CREATE TABLE %s (id bigint, data string, float float) USING iceberg", tableName); | ||
| sql("INSERT INTO %s VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', float('NaN'))", tableName); | ||
| sql("CREATE TABLE %s (id bigint, data string, float float, binary binary) USING iceberg", tableName); | ||
| sql("INSERT INTO %s VALUES (1, 'a', 1.0, X''), (2, 'b', 2.0, X'11'), (3, 'c', float('NaN'), X'1111')", tableName); | ||
|
||
|
|
||
| this.scanEventCount = 0; | ||
| this.lastScanEvent = null; | ||
|
|
@@ -64,14 +64,16 @@ public void removeTables() { | |
| @Test | ||
| public void testSelect() { | ||
| List<Object[]> expected = ImmutableList.of( | ||
| row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN)); | ||
| row(1L, "a", 1.0F, new byte[]{}), | ||
| row(2L, "b", 2.0F, new byte[]{0x11}), | ||
| row(3L, "c", Float.NaN, new byte[]{0x11, 0x11})); | ||
|
|
||
| assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); | ||
| } | ||
|
|
||
| @Test | ||
| public void testSelectRewrite() { | ||
| List<Object[]> expected = ImmutableList.of(row(3L, "c", Float.NaN)); | ||
| List<Object[]> expected = ImmutableList.of(row(3L, "c", Float.NaN, new byte[]{0x11, 0x11})); | ||
|
|
||
| assertEquals("Should return all expected rows", expected, | ||
| sql("SELECT * FROM %s where float = float('NaN')", tableName)); | ||
|
|
@@ -120,4 +122,12 @@ public void testMetadataTables() { | |
| ImmutableList.of(row(ANY, ANY, null, "append", ANY, ANY)), | ||
| sql("SELECT * FROM %s.snapshots", tableName)); | ||
| } | ||
|
|
||
| @Test | ||
| public void testFilterBinary() { | ||
| List<Object[]> expected = ImmutableList.of(row(3L, "c", Float.NaN, new byte[]{0x11, 0x11})); | ||
|
|
||
| assertEquals("Should return all expected rows", expected, | ||
| sql("SELECT * FROM %s where binary > X'1101'", tableName)); | ||
| } | ||
| } | ||
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.
There's an existing ByteBuffers utility class, org.apache.iceberg.util.ByteBuffers, which you can use here. Can you please use that for consistency? It's also more efficient in some cases (doesn't always necessarily allocate, handles offsets more thoroughly, etc).
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.
Thanks for your review, I have modified the code