-
Notifications
You must be signed in to change notification settings - Fork 1.7k
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
[Kernel] Default Parquet reader implementation #1846
Closed
Closed
Changes from 3 commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
fdcc4c6
[Kernel] Default Parquet reader implementation
vkorukanti a17bff9
small issue
vkorukanti 4001179
Update DefaultColumnarBatch.java
vkorukanti aabccb3
address review
vkorukanti a37ba8f
fix
vkorukanti 7a31155
fix data test issue
vkorukanti 2de16c8
use closeableiter.map
vkorukanti 8d87de3
fix
vkorukanti 9dffd61
Update kernel/kernel-default/src/main/java/io/delta/kernel/DefaultKer…
vkorukanti File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
187 changes: 187 additions & 0 deletions
187
kernel/kernel-default/src/main/java/io/delta/kernel/DefaultKernelUtils.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,187 @@ | ||
/* | ||
* Copyright (2023) The Delta Lake Project Authors. | ||
* | ||
* Licensed 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 io.delta.kernel; | ||
|
||
import java.util.ArrayList; | ||
import java.util.List; | ||
import org.apache.parquet.schema.GroupType; | ||
import org.apache.parquet.schema.MessageType; | ||
import org.apache.parquet.schema.Type; | ||
|
||
import io.delta.kernel.types.DataType; | ||
import io.delta.kernel.types.StructField; | ||
import io.delta.kernel.types.StructType; | ||
|
||
public class DefaultKernelUtils | ||
{ | ||
private DefaultKernelUtils() {} | ||
|
||
/** | ||
* Given the file schema in Parquet file and selected columns by Delta, return | ||
* a subschema of the file schema. | ||
* | ||
* @param fileSchema | ||
* @param deltaType | ||
* @return | ||
*/ | ||
public static final MessageType pruneSchema( | ||
MessageType fileSchema, // parquet | ||
StructType deltaType) // delta-core | ||
{ | ||
return deltaType.fields().stream() | ||
.map(column -> { | ||
Type type = findStructField(fileSchema, column); | ||
if (type == null) { | ||
return null; | ||
} | ||
Type prunedSubfields = pruneSubfields(type, column.getDataType()); | ||
return new MessageType(column.getName(), prunedSubfields); | ||
}) | ||
.filter(type -> type != null) | ||
.reduce(MessageType::union) | ||
.get(); | ||
} | ||
|
||
private static Type findStructField(MessageType fileSchema, StructField column) | ||
{ | ||
// TODO: Need a way to search by id once we start supporting column mapping `id` mode. | ||
final String columnName = column.getName(); | ||
if (fileSchema.containsField(columnName)) { | ||
return fileSchema.getType(columnName); | ||
} | ||
// Parquet is case-sensitive, but hive is not. all hive columns get converted to lowercase | ||
// check for direct match above but if no match found, try case-insensitive match | ||
for (org.apache.parquet.schema.Type type : fileSchema.getFields()) { | ||
if (type.getName().equalsIgnoreCase(columnName)) { | ||
return type; | ||
} | ||
} | ||
|
||
return null; | ||
} | ||
|
||
private static Type pruneSubfields(Type type, DataType deltaDatatype) | ||
vkorukanti marked this conversation as resolved.
Show resolved
Hide resolved
|
||
{ | ||
if (!(deltaDatatype instanceof StructType)) { | ||
// there is no pruning for non-struct types | ||
return type; | ||
} | ||
|
||
GroupType groupType = (GroupType) type; | ||
StructType deltaStructType = (StructType) deltaDatatype; | ||
List<Type> newParquetSubFields = new ArrayList<>(); | ||
for (StructField subField : deltaStructType.fields()) { | ||
String subFieldName = subField.getName(); | ||
Type parquetSubFieldType = groupType.getType(subFieldName); | ||
if (parquetSubFieldType == null) { | ||
for (org.apache.parquet.schema.Type typeTemp : groupType.getFields()) { | ||
if (typeTemp.getName().equalsIgnoreCase(subFieldName)) { | ||
parquetSubFieldType = type; | ||
} | ||
} | ||
} | ||
newParquetSubFields.add(parquetSubFieldType); | ||
} | ||
return groupType.withNewFields(newParquetSubFields); | ||
} | ||
|
||
/** | ||
* Precondition-style validation that throws {@link IllegalArgumentException}. | ||
* | ||
* @param isValid {@code true} if valid, {@code false} if an exception should be thrown | ||
* @throws IllegalArgumentException if {@code isValid} is false | ||
*/ | ||
public static void checkArgument(boolean isValid) | ||
throws IllegalArgumentException | ||
{ | ||
if (!isValid) { | ||
throw new IllegalArgumentException(); | ||
} | ||
} | ||
|
||
/** | ||
* Precondition-style validation that throws {@link IllegalArgumentException}. | ||
* | ||
* @param isValid {@code true} if valid, {@code false} if an exception should be thrown | ||
* @param message A String message for the exception. | ||
* @throws IllegalArgumentException if {@code isValid} is false | ||
*/ | ||
public static void checkArgument(boolean isValid, String message) | ||
throws IllegalArgumentException | ||
{ | ||
if (!isValid) { | ||
throw new IllegalArgumentException(message); | ||
} | ||
} | ||
|
||
/** | ||
* Precondition-style validation that throws {@link IllegalArgumentException}. | ||
* | ||
* @param isValid {@code true} if valid, {@code false} if an exception should be thrown | ||
* @param message A String message for the exception. | ||
* @param args Objects used to fill in {@code %s} placeholders in the message | ||
* @throws IllegalArgumentException if {@code isValid} is false | ||
*/ | ||
public static void checkArgument(boolean isValid, String message, Object... args) | ||
throws IllegalArgumentException | ||
{ | ||
if (!isValid) { | ||
throw new IllegalArgumentException( | ||
String.format(String.valueOf(message), args)); | ||
} | ||
} | ||
|
||
/** | ||
* Precondition-style validation that throws {@link IllegalStateException}. | ||
* | ||
* @param isValid {@code true} if valid, {@code false} if an exception should be thrown | ||
* @param message A String message for the exception. | ||
* @throws IllegalStateException if {@code isValid} is false | ||
*/ | ||
public static void checkState(boolean isValid, String message) | ||
throws IllegalStateException | ||
{ | ||
if (!isValid) { | ||
throw new IllegalStateException(message); | ||
} | ||
} | ||
|
||
/** | ||
* Search for the Parquet type for in the {@code groupType} for the field equilant to | ||
* {@code field}. | ||
* | ||
* @param groupType Parquet group type coming from the file schema. | ||
* @param field Sub field given as Delta Kernel's {@link StructField} | ||
* @return {@link Type} of the Parquet field. Returns {@code null}, if not found. | ||
*/ | ||
public static Type findFieldType(GroupType groupType, StructField field) | ||
vkorukanti marked this conversation as resolved.
Show resolved
Hide resolved
|
||
{ | ||
// TODO: Need a way to search by id once we start supporting column mapping `id` mode. | ||
final String columnName = field.getName(); | ||
if (groupType.containsField(columnName)) { | ||
return groupType.getType(columnName); | ||
} | ||
// Parquet is case-sensitive, but hive is not. all hive columns get converted to lowercase | ||
// check for direct match above but if no match found, try case-insensitive match | ||
for (org.apache.parquet.schema.Type type : groupType.getFields()) { | ||
if (type.getName().equalsIgnoreCase(columnName)) { | ||
return type; | ||
} | ||
} | ||
|
||
return null; | ||
} | ||
} |
133 changes: 133 additions & 0 deletions
133
kernel/kernel-default/src/main/java/io/delta/kernel/client/DefaultParquetHandler.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,133 @@ | ||
/* | ||
* Copyright (2023) The Delta Lake Project Authors. | ||
* | ||
* Licensed 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 io.delta.kernel.client; | ||
|
||
import java.io.IOException; | ||
import org.apache.hadoop.conf.Configuration; | ||
|
||
import io.delta.kernel.data.ColumnarBatch; | ||
import io.delta.kernel.data.FileDataReadResult; | ||
import io.delta.kernel.data.Row; | ||
import io.delta.kernel.expressions.Expression; | ||
import io.delta.kernel.fs.FileStatus; | ||
import io.delta.kernel.parquet.ParquetBatchReader; | ||
import io.delta.kernel.types.StructType; | ||
import io.delta.kernel.utils.CloseableIterator; | ||
import io.delta.kernel.utils.Utils; | ||
|
||
public class DefaultParquetHandler | ||
implements ParquetHandler | ||
{ | ||
private final Configuration hadoopConf; | ||
|
||
public DefaultParquetHandler(Configuration hadoopConf) | ||
{ | ||
this.hadoopConf = hadoopConf; | ||
} | ||
|
||
@Override | ||
public CloseableIterator<FileReadContext> contextualizeFileReads( | ||
CloseableIterator<Row> fileIter, | ||
Expression predicate) | ||
{ | ||
return new CloseableIterator<FileReadContext>() | ||
{ | ||
@Override | ||
public void close() | ||
throws IOException | ||
{ | ||
fileIter.close(); | ||
} | ||
|
||
@Override | ||
public boolean hasNext() | ||
{ | ||
return fileIter.hasNext(); | ||
} | ||
|
||
@Override | ||
public FileReadContext next() | ||
{ | ||
return () -> fileIter.next(); | ||
vkorukanti marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
}; | ||
} | ||
|
||
@Override | ||
public CloseableIterator<FileDataReadResult> readParquetFiles( | ||
CloseableIterator<FileReadContext> fileIter, | ||
StructType physicalSchema) throws IOException | ||
{ | ||
return new CloseableIterator<FileDataReadResult>() | ||
{ | ||
private FileReadContext currentFile; | ||
private CloseableIterator<ColumnarBatch> currentFileReader; | ||
|
||
@Override | ||
public void close() | ||
throws IOException | ||
{ | ||
if (currentFileReader != null) { | ||
currentFileReader.close(); | ||
} | ||
|
||
fileIter.close(); | ||
// TODO: implement safe close of multiple closeables. | ||
} | ||
|
||
@Override | ||
public boolean hasNext() | ||
{ | ||
// There is no file in reading or the current file being read has no more data | ||
// initialize the next file reader or return false if there are no more files to | ||
// read. | ||
if (currentFileReader == null || !currentFileReader.hasNext()) { | ||
if (fileIter.hasNext()) { | ||
currentFile = fileIter.next(); | ||
FileStatus fileStatus = Utils.getFileStatus(currentFile.getScanFileRow()); | ||
ParquetBatchReader batchReader = new ParquetBatchReader(hadoopConf); | ||
vkorukanti marked this conversation as resolved.
Show resolved
Hide resolved
|
||
currentFileReader = batchReader.read(fileStatus.getPath(), physicalSchema); | ||
} | ||
else { | ||
return false; | ||
} | ||
} | ||
|
||
return currentFileReader.hasNext(); | ||
} | ||
|
||
@Override | ||
public FileDataReadResult next() | ||
{ | ||
final ColumnarBatch data = currentFileReader.next(); | ||
return new FileDataReadResult() | ||
{ | ||
@Override | ||
public ColumnarBatch getData() | ||
{ | ||
return data; | ||
} | ||
|
||
@Override | ||
public Row getScanFileRow() | ||
{ | ||
return currentFile.getScanFileRow(); | ||
} | ||
}; | ||
} | ||
}; | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Seems like this and
pruneSchema
can use the same code if we generalize it forGroupType
?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.
Cleaned up a bit, but not clean to merge them. MessageType and Group type needs two different inputs. For now keeping them separate. It only adds extra 5lines of code.
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.
This only prunes the 2nd level of nesting right?
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.
Can't we still factor out the shared code?
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.
You need the MessageType for the
pruneSchema
.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.
MessageType extends GroupType through right? So pruneSchema can just call a helper. Not a big deal though, can we just either fix the pruning for nested fields or add a note that nested schema pruning is only 2nd level?