Skip to content
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

[Spark][MR]: Reuse InputFilesDecryptor #4595

Closed
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 @@ -57,8 +57,7 @@
import org.apache.iceberg.data.avro.DataReader;
import org.apache.iceberg.data.orc.GenericOrcReader;
import org.apache.iceberg.data.parquet.GenericParquetReaders;
import org.apache.iceberg.encryption.EncryptedFiles;
import org.apache.iceberg.encryption.EncryptionManager;
import org.apache.iceberg.encryption.InputFilesDecryptor;
import org.apache.iceberg.expressions.Evaluator;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.Expressions;
Expand Down Expand Up @@ -208,7 +207,7 @@ private static final class IcebergRecordReader<T> extends RecordReader<Void, T>
private T current;
private CloseableIterator<T> currentIterator;
private FileIO io;
private EncryptionManager encryptionManager;
private InputFilesDecryptor inputFilesDecryptor;

@Override
public void initialize(InputSplit split, TaskAttemptContext newContext) {
Expand All @@ -219,7 +218,7 @@ public void initialize(InputSplit split, TaskAttemptContext newContext) {
Table table = ((IcebergSplit) split).table();
HiveIcebergStorageHandler.checkAndSetIoConfig(conf, table);
this.io = table.io();
this.encryptionManager = table.encryption();
this.inputFilesDecryptor = new InputFilesDecryptor(task, io, table.encryption());
this.tasks = task.files().iterator();
this.tableSchema = InputFormatConfig.tableSchema(conf);
this.nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING);
Expand Down Expand Up @@ -275,9 +274,7 @@ public void close() throws IOException {

private CloseableIterable<T> openTask(FileScanTask currentTask, Schema readSchema) {
DataFile file = currentTask.file();
InputFile inputFile = encryptionManager.decrypt(EncryptedFiles.encryptedInput(
io.newInputFile(file.path().toString()),
file.keyMetadata()));
InputFile inputFile = inputFilesDecryptor.getInputFile(currentTask);

CloseableIterable<T> iterable;
switch (file.format()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.stream.Stream;
import org.apache.avro.generic.GenericData;
import org.apache.avro.util.Utf8;
import org.apache.iceberg.CombinedScanTask;
Expand All @@ -36,13 +35,10 @@
import org.apache.iceberg.Schema;
import org.apache.iceberg.StructLike;
import org.apache.iceberg.Table;
import org.apache.iceberg.encryption.EncryptedFiles;
import org.apache.iceberg.encryption.EncryptedInputFile;
import org.apache.iceberg.encryption.InputFilesDecryptor;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types.NestedField;
import org.apache.iceberg.types.Types.StructType;
Expand All @@ -65,7 +61,7 @@ abstract class BaseDataReader<T> implements Closeable {

private final Table table;
private final Iterator<FileScanTask> tasks;
private final Map<String, InputFile> inputFiles;
private final InputFilesDecryptor inputFilesDecryptor;

private CloseableIterator<T> currentIterator;
private T current = null;
Expand All @@ -74,20 +70,7 @@ abstract class BaseDataReader<T> implements Closeable {
BaseDataReader(Table table, CombinedScanTask task) {
this.table = table;
this.tasks = task.files().iterator();
Map<String, ByteBuffer> keyMetadata = Maps.newHashMap();
task.files().stream()
.flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
.forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata()));
Stream<EncryptedInputFile> encrypted = keyMetadata.entrySet().stream()
.map(entry -> EncryptedFiles.encryptedInput(table.io().newInputFile(entry.getKey()), entry.getValue()));

// decrypt with the batch call to avoid multiple RPCs to a key server, if possible
Iterable<InputFile> decryptedFiles = table.encryption().decrypt(encrypted::iterator);

Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(task.files().size());
decryptedFiles.forEach(decrypted -> files.putIfAbsent(decrypted.location(), decrypted));
this.inputFiles = ImmutableMap.copyOf(files);

this.inputFilesDecryptor = new InputFilesDecryptor(task, table().io(), table.encryption());
this.currentIterator = CloseableIterator.empty();
}

Expand Down Expand Up @@ -139,11 +122,11 @@ public void close() throws IOException {

protected InputFile getInputFile(FileScanTask task) {
Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
return inputFiles.get(task.file().path().toString());
return inputFilesDecryptor.getInputFile(task);
}

protected InputFile getInputFile(String location) {
return inputFiles.get(location);
return inputFilesDecryptor.getInputFile(location);
}

protected Map<Integer, ?> constantsMap(FileScanTask task, Schema readSchema) {
Expand Down