Skip to content
Merged
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 @@ -47,6 +47,8 @@
import java.util.Queue;
import java.util.function.BiFunction;

import static com.google.common.base.Preconditions.checkArgument;
import static io.trino.plugin.iceberg.IcebergMetadataColumn.isMetadataColumnId;
import static io.trino.plugin.iceberg.util.Timestamps.timestampTzToMicros;
import static io.trino.spi.type.TimeType.TIME_MICROS;
import static io.trino.spi.type.TimestampType.TIMESTAMP_MICROS;
Expand Down Expand Up @@ -84,6 +86,7 @@ public static Expression toIcebergExpression(TupleDomain<IcebergColumnHandle> tu
List<Expression> conjuncts = new ArrayList<>();
for (Map.Entry<IcebergColumnHandle, Domain> entry : domainMap.entrySet()) {
IcebergColumnHandle columnHandle = entry.getKey();
checkArgument(!isMetadataColumnId(columnHandle.getId()), "Constraint on an unexpected column %s", columnHandle);
Domain domain = entry.getValue();
conjuncts.add(toIcebergExpression(columnHandle.getQualifiedName(), columnHandle.getType(), domain));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1730,16 +1730,23 @@ public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(C
Map<IcebergColumnHandle, Domain> newUnenforced = new LinkedHashMap<>();
Map<IcebergColumnHandle, Domain> domains = predicate.getDomains().orElseThrow(() -> new VerifyException("No domains"));
domains.forEach((columnHandle, domain) -> {
// Iceberg metadata columns can not be used to filter a table scan in Iceberg library
// TODO (https://github.com/trinodb/trino/issues/8759) structural types cannot be used to filter a table scan in Iceberg library.
if (isMetadataColumnId(columnHandle.getId()) || isStructuralType(columnHandle.getType()) ||
if (isStructuralType(columnHandle.getType()) ||
// Iceberg orders UUID values differently than Trino (perhaps due to https://bugs.openjdk.org/browse/JDK-7025832), so allow only IS NULL / IS NOT NULL checks
(columnHandle.getType() == UUID && !(domain.isOnlyNull() || domain.getValues().isAll()))) {
unsupported.put(columnHandle, domain);
}
else if (canEnforceColumnConstraintInSpecs(typeOperators, icebergTable, partitionSpecIds, columnHandle, domain)) {
newEnforced.put(columnHandle, domain);
}
else if (isMetadataColumnId(columnHandle.getId())) {
if (columnHandle.isPathColumn()) {
newEnforced.put(columnHandle, domain);
}
else {
unsupported.put(columnHandle, domain);
}
}
else {
newUnenforced.put(columnHandle, domain);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,8 @@

import java.io.IOException;
import java.io.UncheckedIOException;
import java.net.URI;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should "Return $path without URL encoding in Iceberg" commit have any test changes/additions?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ideally it should, but let me handle in #13457

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

url encoding (or lack of) should be exercisable independently from double slashes.
eg path containing %.

Copy link
Copy Markdown
Member Author

@ebyhr ebyhr Aug 3, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not sure if I understood your suggestion correctly, but just adding % to file or directory name wouldn't work because it doesn't pass in !path.equals(hadoopPath.toString()) in hadoopPath().

import java.net.URLEncoder;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
Expand Down Expand Up @@ -145,6 +147,7 @@
import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_CANNOT_OPEN_SPLIT;
import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_CURSOR_ERROR;
import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_FILESYSTEM_ERROR;
import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_INVALID_METADATA;
import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_MISSING_DATA;
import static io.trino.plugin.iceberg.IcebergMetadataColumn.FILE_MODIFIED_TIME;
import static io.trino.plugin.iceberg.IcebergMetadataColumn.FILE_PATH;
Expand Down Expand Up @@ -179,6 +182,7 @@
import static io.trino.spi.type.VarbinaryType.VARBINARY;
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.String.format;
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Locale.ENGLISH;
import static java.util.Objects.requireNonNull;
import static java.util.function.Predicate.not;
Expand Down Expand Up @@ -319,7 +323,7 @@ public ConnectorPageSource createPageSource(
ReaderPageSource dataPageSource = createDataPageSource(
session,
hdfsContext,
new Path(split.getPath()),
split.getPath(),
split.getStart(),
split.getLength(),
fileSize,
Expand Down Expand Up @@ -448,7 +452,7 @@ private ConnectorPageSource openDeletes(
return createDataPageSource(
session,
new HdfsContext(session),
new Path(delete.path().toString()),
delete.path().toString(),
0,
delete.fileSizeInBytes(),
delete.fileSizeInBytes(),
Expand All @@ -465,7 +469,7 @@ private ConnectorPageSource openDeletes(
public ReaderPageSource createDataPageSource(
ConnectorSession session,
HdfsContext hdfsContext,
Path path,
String path,
long start,
long length,
long fileSize,
Expand All @@ -477,13 +481,15 @@ public ReaderPageSource createDataPageSource(
Optional<NameMapping> nameMapping,
Map<Integer, Optional<String>> partitionKeys)
{
Path hadoopPath = new Path(hadoopPath(path));
switch (fileFormat) {
case ORC:
return createOrcPageSource(
hdfsEnvironment,
session.getIdentity(),
hdfsEnvironment.getConfiguration(hdfsContext, path),
hdfsEnvironment.getConfiguration(hdfsContext, hadoopPath),
path,
hadoopPath,
start,
length,
fileSize,
Expand All @@ -507,8 +513,9 @@ public ReaderPageSource createDataPageSource(
return createParquetPageSource(
hdfsEnvironment,
session.getIdentity(),
hdfsEnvironment.getConfiguration(hdfsContext, path),
hdfsEnvironment.getConfiguration(hdfsContext, hadoopPath),
path,
hadoopPath,
start,
length,
fileSize,
Expand All @@ -524,6 +531,7 @@ public ReaderPageSource createDataPageSource(
return createAvroPageSource(
fileIoProvider.createFileIo(hdfsContext, session.getQueryId()),
path,
hadoopPath,
start,
length,
fileModifiedTime,
Expand All @@ -539,7 +547,8 @@ private static ReaderPageSource createOrcPageSource(
HdfsEnvironment hdfsEnvironment,
ConnectorIdentity identity,
Configuration configuration,
Path path,
String path,
Path hadoopPath,
long start,
long length,
long fileSize,
Expand All @@ -554,10 +563,10 @@ private static ReaderPageSource createOrcPageSource(
{
OrcDataSource orcDataSource = null;
try {
FileSystem fileSystem = hdfsEnvironment.getFileSystem(identity, path, configuration);
FSDataInputStream inputStream = hdfsEnvironment.doAs(identity, () -> fileSystem.open(path));
FileSystem fileSystem = hdfsEnvironment.getFileSystem(identity, hadoopPath, configuration);
FSDataInputStream inputStream = hdfsEnvironment.doAs(identity, () -> fileSystem.open(hadoopPath));
orcDataSource = new HdfsOrcDataSource(
new OrcDataSourceId(path.toString()),
new OrcDataSourceId(hadoopPath.toString()),
fileSize,
options,
inputStream,
Expand Down Expand Up @@ -608,7 +617,7 @@ else if (partitionKeys.containsKey(column.getId())) {
deserializePartitionValue(trinoType, partitionKeys.get(column.getId()).orElse(null), column.getName()))));
}
else if (column.isPathColumn()) {
columnAdaptations.add(ColumnAdaptation.constantColumn(nativeValueToBlock(FILE_PATH.getType(), utf8Slice(path.toString()))));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Q are we changing the behavior here? i.e. before this PR, effectively hadoopPath() was being returned here since that was propagated from the split, but now it's the actual path without encoding.

Copy link
Copy Markdown
Member Author

@ebyhr ebyhr Jul 11, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, it's changing. I will mention in a release note. Relates to #13012 (comment)

columnAdaptations.add(ColumnAdaptation.constantColumn(nativeValueToBlock(FILE_PATH.getType(), utf8Slice(path))));
}
else if (column.isFileModifiedTimeColumn()) {
columnAdaptations.add(ColumnAdaptation.constantColumn(nativeValueToBlock(FILE_MODIFIED_TIME.getType(), packDateTimeWithZone(fileModifiedTime.orElseThrow(), UTC_KEY))));
Expand Down Expand Up @@ -690,7 +699,7 @@ else if (orcColumn != null) {
if (e instanceof TrinoException) {
throw (TrinoException) e;
}
String message = format("Error opening Iceberg split %s (offset=%s, length=%s): %s", path, start, length, e.getMessage());
String message = format("Error opening Iceberg split %s (offset=%s, length=%s): %s", hadoopPath, start, length, e.getMessage());
if (e instanceof BlockMissingException) {
throw new TrinoException(ICEBERG_MISSING_DATA, message, e);
}
Expand Down Expand Up @@ -886,7 +895,8 @@ private static ReaderPageSource createParquetPageSource(
HdfsEnvironment hdfsEnvironment,
ConnectorIdentity identity,
Configuration configuration,
Path path,
String path,
Path hadoopPath,
long start,
long length,
long fileSize,
Expand All @@ -902,9 +912,9 @@ private static ReaderPageSource createParquetPageSource(

ParquetDataSource dataSource = null;
try {
FileSystem fileSystem = hdfsEnvironment.getFileSystem(identity, path, configuration);
FSDataInputStream inputStream = hdfsEnvironment.doAs(identity, () -> fileSystem.open(path));
dataSource = new HdfsParquetDataSource(new ParquetDataSourceId(path.toString()), fileSize, inputStream, fileFormatDataSourceStats, options);
FileSystem fileSystem = hdfsEnvironment.getFileSystem(identity, hadoopPath, configuration);
FSDataInputStream inputStream = hdfsEnvironment.doAs(identity, () -> fileSystem.open(hadoopPath));
dataSource = new HdfsParquetDataSource(new ParquetDataSourceId(hadoopPath.toString()), fileSize, inputStream, fileFormatDataSourceStats, options);
ParquetDataSource theDataSource = dataSource; // extra variable required for lambda below
ParquetMetadata parquetMetadata = hdfsEnvironment.doAs(identity, () -> MetadataReader.readFooter(theDataSource));
FileMetaData fileMetaData = parquetMetadata.getFileMetaData();
Expand Down Expand Up @@ -975,7 +985,7 @@ else if (partitionKeys.containsKey(column.getId())) {
deserializePartitionValue(trinoType, partitionKeys.get(column.getId()).orElse(null), column.getName())));
}
else if (column.isPathColumn()) {
constantPopulatingPageSourceBuilder.addConstantColumn(nativeValueToBlock(FILE_PATH.getType(), utf8Slice(path.toString())));
constantPopulatingPageSourceBuilder.addConstantColumn(nativeValueToBlock(FILE_PATH.getType(), utf8Slice(path)));
}
else if (column.isFileModifiedTimeColumn()) {
constantPopulatingPageSourceBuilder.addConstantColumn(nativeValueToBlock(FILE_MODIFIED_TIME.getType(), packDateTimeWithZone(fileModifiedTime.orElseThrow(), UTC_KEY)));
Expand Down Expand Up @@ -1033,7 +1043,7 @@ else if (column.isRowPositionColumn()) {
if (e instanceof TrinoException) {
throw (TrinoException) e;
}
String message = format("Error opening Iceberg split %s (offset=%s, length=%s): %s", path, start, length, e.getMessage());
String message = format("Error opening Iceberg split %s (offset=%s, length=%s): %s", hadoopPath, start, length, e.getMessage());

if (e instanceof ParquetCorruptionException) {
throw new TrinoException(ICEBERG_BAD_DATA, message, e);
Expand All @@ -1048,7 +1058,8 @@ else if (column.isRowPositionColumn()) {

private ReaderPageSource createAvroPageSource(
FileIO fileIo,
Path path,
String path,
Path hadoopPath,
long start,
long length,
OptionalLong fileModifiedTime,
Expand All @@ -1066,7 +1077,7 @@ private ReaderPageSource createAvroPageSource(
.orElse(columns);

// The column orders in the generated schema might be different from the original order
try (DataFileStream<GenericRecord> avroFileReader = new DataFileStream<>(fileIo.newInputFile(path.toString()).newStream(), new GenericDatumReader<>())) {
try (DataFileStream<GenericRecord> avroFileReader = new DataFileStream<>(fileIo.newInputFile(hadoopPath.toString()).newStream(), new GenericDatumReader<>())) {
org.apache.avro.Schema avroSchema = avroFileReader.getSchema();
List<org.apache.avro.Schema.Field> fileFields = avroSchema.getFields();
if (nameMapping.isPresent() && fileFields.stream().noneMatch(IcebergPageSourceProvider::hasId)) {
Expand All @@ -1086,7 +1097,7 @@ private ReaderPageSource createAvroPageSource(
org.apache.avro.Schema.Field field = fileColumnsByIcebergId.get(column.getId());

if (column.isPathColumn()) {
constantPopulatingPageSourceBuilder.addConstantColumn(nativeValueToBlock(FILE_PATH.getType(), utf8Slice(path.toString())));
constantPopulatingPageSourceBuilder.addConstantColumn(nativeValueToBlock(FILE_PATH.getType(), utf8Slice(path)));
}
else if (column.isFileModifiedTimeColumn()) {
constantPopulatingPageSourceBuilder.addConstantColumn(nativeValueToBlock(FILE_MODIFIED_TIME.getType(), packDateTimeWithZone(fileModifiedTime.orElseThrow(), UTC_KEY)));
Expand Down Expand Up @@ -1114,7 +1125,7 @@ else if (field == null) {
return new ReaderPageSource(
constantPopulatingPageSourceBuilder.build(new IcebergAvroPageSource(
fileIo,
path.toString(),
hadoopPath.toString(),
start,
length,
fileSchema,
Expand Down Expand Up @@ -1291,4 +1302,18 @@ private static TrinoException handleException(OrcDataSourceId dataSourceId, Exce
}
return new TrinoException(ICEBERG_CURSOR_ERROR, format("Failed to read ORC file: %s", dataSourceId), exception);
}

private static String hadoopPath(String path)
{
// hack to preserve the original path for S3 if necessary
Path hadoopPath = new Path(path);
if ("s3".equals(hadoopPath.toUri().getScheme()) && !path.equals(hadoopPath.toString())) {
if (hadoopPath.toUri().getFragment() != null) {
throw new TrinoException(ICEBERG_INVALID_METADATA, "Unexpected URI fragment in path: " + path);
}
URI uri = URI.create(path);
return uri + "#" + URLEncoder.encode(uri.getPath(), UTF_8);
}
return path;
}
}
Loading