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 @@ -16,15 +16,26 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iceberg.spark.extensions;
package org.apache.iceberg.spark.source;

import static org.assertj.core.api.Assertions.assertThat;

import java.util.function.Supplier;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.ParameterizedTestExtension;
import org.apache.iceberg.Parameters;
import org.apache.iceberg.Table;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.rest.RESTCatalog;
import org.apache.iceberg.rest.RESTCatalogProperties;
import org.apache.iceberg.spark.SparkCatalogConfig;
import org.apache.iceberg.spark.sql.TestSelect;
import org.apache.spark.sql.connector.read.Batch;
import org.apache.spark.sql.util.CaseInsensitiveStringMap;
import org.assertj.core.api.InstanceOfAssertFactories;
import org.junit.jupiter.api.TestTemplate;
import org.junit.jupiter.api.extension.ExtendWith;

@ExtendWith(ParameterizedTestExtension.class)
Expand All @@ -38,8 +49,6 @@ protected static Object[][] parameters() {
ImmutableMap.builder()
.putAll(SparkCatalogConfig.REST.properties())
.put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI))
// this flag is typically only set by the server, but we set it from the client for
// testing
.put(
RESTCatalogProperties.SCAN_PLANNING_MODE,
RESTCatalogProperties.ScanPlanningMode.SERVER.modeName())
Expand All @@ -48,4 +57,36 @@ protected static Object[][] parameters() {
}
};
}

@TestTemplate
public void fileIOIsPropagated() {
RESTCatalog catalog = new RESTCatalog();
catalog.setConf(new Configuration());
catalog.initialize(
"test",
ImmutableMap.<String, String>builder()
.putAll(restCatalog.properties())
.put(
RESTCatalogProperties.SCAN_PLANNING_MODE,
RESTCatalogProperties.ScanPlanningMode.SERVER.modeName())
.build());
Table table = catalog.loadTable(tableIdent);

SparkScanBuilder builder = new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty());
verifyFileIOHasPlanId(builder.build().toBatch(), table);
verifyFileIOHasPlanId(builder.buildCopyOnWriteScan().toBatch(), table);
}

private void verifyFileIOHasPlanId(Batch batch, Table table) {
FileIO fileIOForScan =
(FileIO)
assertThat(batch)
.extracting("fileIO")
.isInstanceOf(Supplier.class)
.asInstanceOf(InstanceOfAssertFactories.type(Supplier.class))
.actual()
.get();
assertThat(fileIOForScan.properties()).containsKey(RESTCatalogProperties.REST_SCAN_PLAN_ID);
assertThat(table.io().properties()).doesNotContainKey(RESTCatalogProperties.REST_SCAN_PLAN_ID);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.iceberg.formats.FormatModelRegistry;
import org.apache.iceberg.formats.ReadBuilder;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.iceberg.spark.OrcBatchReadConf;
Expand All @@ -51,13 +52,14 @@ abstract class BaseBatchReader<T extends ScanTask> extends BaseReader<ColumnarBa

BaseBatchReader(
Table table,
FileIO fileIO,
ScanTaskGroup<T> taskGroup,
Schema expectedSchema,
boolean caseSensitive,
ParquetBatchReadConf parquetConf,
OrcBatchReadConf orcConf,
boolean cacheDeleteFilesOnExecutors) {
super(table, taskGroup, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors);
super(table, fileIO, taskGroup, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors);
this.parquetConf = parquetConf;
this.orcConf = orcConf;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
import org.apache.iceberg.deletes.DeleteCounter;
import org.apache.iceberg.encryption.EncryptingFileIO;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.mapping.NameMapping;
import org.apache.iceberg.mapping.NameMappingParser;
Expand All @@ -68,6 +69,7 @@ abstract class BaseReader<T, TaskT extends ScanTask> implements Closeable {
private static final Logger LOG = LoggerFactory.getLogger(BaseReader.class);

private final Table table;
private final EncryptingFileIO fileIO;
private final Schema expectedSchema;
private final boolean caseSensitive;
private final NameMapping nameMapping;
Expand All @@ -83,11 +85,13 @@ abstract class BaseReader<T, TaskT extends ScanTask> implements Closeable {

BaseReader(
Table table,
FileIO fileIO,
ScanTaskGroup<TaskT> taskGroup,
Schema expectedSchema,
boolean caseSensitive,
boolean cacheDeleteFilesOnExecutors) {
this.table = table;
this.fileIO = EncryptingFileIO.combine(fileIO, table().encryption());
this.taskGroup = taskGroup;
this.tasks = taskGroup.tasks().iterator();
this.currentIterator = CloseableIterator.empty();
Expand Down Expand Up @@ -179,9 +183,8 @@ protected InputFile getInputFile(String location) {
private Map<String, InputFile> inputFiles() {
if (lazyInputFiles == null) {
this.lazyInputFiles =
EncryptingFileIO.combine(table().io(), table().encryption())
.bulkDecrypt(
() -> taskGroup.tasks().stream().flatMap(this::referencedFiles).iterator());
fileIO.bulkDecrypt(
() -> taskGroup.tasks().stream().flatMap(this::referencedFiles).iterator());
}

return lazyInputFiles;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,17 +28,19 @@
import org.apache.iceberg.formats.FormatModelRegistry;
import org.apache.iceberg.formats.ReadBuilder;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.InputFile;
import org.apache.spark.sql.catalyst.InternalRow;

abstract class BaseRowReader<T extends ScanTask> extends BaseReader<InternalRow, T> {
BaseRowReader(
Table table,
FileIO fileIO,
ScanTaskGroup<T> taskGroup,
Schema expectedSchema,
boolean caseSensitive,
boolean cacheDeleteFilesOnExecutors) {
super(table, taskGroup, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors);
super(table, fileIO, taskGroup, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors);
}

protected CloseableIterable<InternalRow> newIterable(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.iceberg.Schema;
import org.apache.iceberg.Table;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.spark.OrcBatchReadConf;
Expand All @@ -52,6 +53,7 @@ class BatchDataReader extends BaseBatchReader<FileScanTask>
OrcBatchReadConf orcBatchReadConf) {
this(
partition.table(),
partition.io(),
partition.taskGroup(),
partition.projection(),
partition.isCaseSensitive(),
Expand All @@ -62,6 +64,7 @@ class BatchDataReader extends BaseBatchReader<FileScanTask>

BatchDataReader(
Table table,
FileIO fileIO,
ScanTaskGroup<FileScanTask> taskGroup,
Schema expectedSchema,
boolean caseSensitive,
Expand All @@ -70,6 +73,7 @@ class BatchDataReader extends BaseBatchReader<FileScanTask>
boolean cacheDeleteFilesOnExecutors) {
super(
table,
fileIO,
taskGroup,
expectedSchema,
caseSensitive,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.iceberg.Table;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.spark.rdd.InputFileBlockHolder;
Expand All @@ -50,6 +51,7 @@ class ChangelogRowReader extends BaseRowReader<ChangelogScanTask>
ChangelogRowReader(SparkInputPartition partition) {
this(
partition.table(),
partition.io(),
partition.taskGroup(),
partition.projection(),
partition.isCaseSensitive(),
Expand All @@ -58,12 +60,14 @@ class ChangelogRowReader extends BaseRowReader<ChangelogScanTask>

ChangelogRowReader(
Table table,
FileIO fileIO,
ScanTaskGroup<ChangelogScanTask> taskGroup,
Schema expectedSchema,
boolean caseSensitive,
boolean cacheDeleteFilesOnExecutors) {
super(
table,
fileIO,
taskGroup,
ChangelogUtil.dropChangelogMetadata(expectedSchema),
caseSensitive,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,17 +25,19 @@
import org.apache.iceberg.Schema;
import org.apache.iceberg.Table;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.io.FileIO;
import org.apache.spark.rdd.InputFileBlockHolder;
import org.apache.spark.sql.catalyst.InternalRow;

public class EqualityDeleteRowReader extends RowDataReader {
public EqualityDeleteRowReader(
CombinedScanTask task,
Table table,
FileIO fileIO,
Schema expectedSchema,
boolean caseSensitive,
boolean cacheDeleteFilesOnExecutors) {
super(table, task, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors);
super(table, fileIO, task, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.ExpressionUtil;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.primitives.Ints;
Expand All @@ -48,6 +49,7 @@ class PositionDeletesRowReader extends BaseRowReader<PositionDeletesScanTask>
PositionDeletesRowReader(SparkInputPartition partition) {
this(
partition.table(),
partition.io(),
partition.taskGroup(),
partition.projection(),
partition.isCaseSensitive(),
Expand All @@ -56,12 +58,12 @@ class PositionDeletesRowReader extends BaseRowReader<PositionDeletesScanTask>

PositionDeletesRowReader(
Table table,
FileIO fileIO,
ScanTaskGroup<PositionDeletesScanTask> taskGroup,
Schema expectedSchema,
boolean caseSensitive,
boolean cacheDeleteFilesOnExecutors) {

super(table, taskGroup, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors);
super(table, fileIO, taskGroup, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors);

int numSplits = taskGroup.tasks().size();
LOG.debug("Reading {} position delete file split(s) for table {}", numSplits, table.name());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.iceberg.Table;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.spark.source.metrics.TaskNumDeletes;
Expand All @@ -47,6 +48,7 @@ class RowDataReader extends BaseRowReader<FileScanTask> implements PartitionRead
RowDataReader(SparkInputPartition partition) {
this(
partition.table(),
partition.io(),
partition.taskGroup(),
partition.projection(),
partition.isCaseSensitive(),
Expand All @@ -55,12 +57,13 @@ class RowDataReader extends BaseRowReader<FileScanTask> implements PartitionRead

RowDataReader(
Table table,
FileIO fileIO,
ScanTaskGroup<FileScanTask> taskGroup,
Schema expectedSchema,
boolean caseSensitive,
boolean cacheDeleteFilesOnExecutors) {

super(table, taskGroup, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors);
super(table, fileIO, taskGroup, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors);

numSplits = taskGroup.tasks().size();
LOG.debug("Reading {} file split(s) for table {}", numSplits, table.name());
Expand Down
Loading
Loading