Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -54,13 +54,15 @@
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding;
import org.apache.iceberg.spark.SparkTableUtil.SparkPartition;
import org.apache.iceberg.spark.source.SparkTable;
import org.apache.iceberg.transforms.PartitionSpecVisitor;
import org.apache.iceberg.transforms.SortOrderVisitor;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.ByteBuffers;
import org.apache.iceberg.util.Pair;
import org.apache.iceberg.util.SortOrderUtil;
import org.apache.spark.sql.Dataset;
Expand Down Expand Up @@ -653,7 +655,8 @@ 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);
byte[] bytes = ByteBuffers.toByteArray((ByteBuffer) lit.value());
return "X'" + BaseEncoding.base16().encode(bytes) + "'";
} else {
return lit.value().toString();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,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 instanceof byte[]) {
Assert.assertArrayEquals(newContext, (byte[]) expectedValue, (byte[]) actualValue);
} else {
assertEquals(newContext, (Object[]) expectedValue, (Object[]) actualValue);
}
} else if (expectedValue != ANY) {
Assert.assertEquals(context + " contents should match", expectedValue, actualValue);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
public class TestSelect extends SparkCatalogTestBase {
private int scanEventCount = 0;
private ScanEvent lastScanEvent = null;
private String binaryTableName = tableName("binary_table");

public TestSelect(String catalogName, String implementation, Map<String, String> config) {
super(catalogName, implementation, config);
Expand All @@ -64,6 +65,7 @@ public void createTables() {
@After
public void removeTables() {
sql("DROP TABLE IF EXISTS %s", tableName);
sql("DROP TABLE IF EXISTS %s", binaryTableName);
}

@Test
Expand Down Expand Up @@ -222,4 +224,16 @@ public void testSpecifySnapshotAndTimestamp() {
.collectAsList();
});
}

@Test
public void testBinaryInFilter() {
sql("CREATE TABLE %s (id bigint, binary binary) USING iceberg", binaryTableName);
sql("INSERT INTO %s VALUES (1, X''), (2, X'1111'), (3, X'11')", binaryTableName);
List<Object[]> expected = ImmutableList.of(row(2L, new byte[] {0x11, 0x11}));

assertEquals(
"Should return all expected rows",
expected,
sql("SELECT id, binary FROM %s where binary > X'11'", binaryTableName));
}
}