Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
2 changes: 1 addition & 1 deletion core/src/main/java/com/netflix/iceberg/BaseTable.java
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
* <p>
* This can be extended by providing a {@link TableOperations} to the constructor.
*/
public class BaseTable implements Table, HasTableOperations {
public class BaseTable implements TableWithTableOperations {
private final TableOperations ops;
private final String name;

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 com.netflix.iceberg;

public interface TableWithTableOperations extends Table, HasTableOperations {
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import com.netflix.iceberg.Table;
import com.netflix.iceberg.TableMetadata;
import com.netflix.iceberg.TableOperations;
import com.netflix.iceberg.TableWithTableOperations;
import com.netflix.iceberg.Tables;
import com.netflix.iceberg.exceptions.AlreadyExistsException;
import com.netflix.iceberg.exceptions.NoSuchTableException;
Expand Down Expand Up @@ -57,7 +58,7 @@ public HadoopTables(Configuration conf) {
* @return table implementation
*/
@Override
public Table load(String location) {
public TableWithTableOperations load(String location) {
TableOperations ops = newTableOps(location);
if (ops.current() == null) {
throw new NoSuchTableException("Table does not exist at location: " + location);
Expand All @@ -76,7 +77,7 @@ public Table load(String location) {
* @return newly created table implementation
*/
@Override
public Table create(Schema schema, PartitionSpec spec, Map<String, String> properties,
public TableWithTableOperations create(Schema schema, PartitionSpec spec, Map<String, String> properties,
String location) {
TableOperations ops = newTableOps(location);
if (ops.current() != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.netflix.iceberg.FileFormat;
import com.netflix.iceberg.Schema;
import com.netflix.iceberg.Table;
import com.netflix.iceberg.TableWithTableOperations;
import com.netflix.iceberg.hadoop.HadoopTables;
import com.netflix.iceberg.spark.SparkSchemaUtil;
import com.netflix.iceberg.types.CheckCompatibility;
Expand Down Expand Up @@ -56,16 +57,16 @@ public String shortName() {

@Override
public DataSourceReader createReader(DataSourceOptions options) {
Table table = findTable(options);
return new Reader(table, lazyConf());
TableWithTableOperations table = findTable(options);
return new Reader(table, table.operations().io());
}

@Override
public Optional<DataSourceWriter> createWriter(String jobId, StructType dfStruct, SaveMode mode,
DataSourceOptions options) {
Preconditions.checkArgument(mode == SaveMode.Append, "Save mode %s is not supported", mode);

Table table = findTable(options);
TableWithTableOperations table = findTable(options);

Schema dfSchema = SparkSchemaUtil.convert(table.schema(), dfStruct);
List<String> errors = CheckCompatibility.writeCompatibilityErrors(table.schema(), dfSchema);
Expand All @@ -89,10 +90,10 @@ public Optional<DataSourceWriter> createWriter(String jobId, StructType dfStruct
.toUpperCase(Locale.ENGLISH));
}

return Optional.of(new Writer(table, lazyConf(), format));
return Optional.of(new Writer(table, format, table.operations().io()));
}

protected Table findTable(DataSourceOptions options) {
protected TableWithTableOperations findTable(DataSourceOptions options) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@rdblue this is the most interesting question raised by this patch. What do you think?

Copy link
Contributor

Choose a reason for hiding this comment

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

What about adding io to the Table interface? I'd rather do that since FileIO is a public interface. I think that is mostly what HasTableOperations is used for anyway.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think that's fine.

Optional<String> location = options.get("path");
Preconditions.checkArgument(location.isPresent(),
"Cannot open table without a location: path is not set");
Expand Down
41 changes: 19 additions & 22 deletions spark/src/main/java/com/netflix/iceberg/spark/source/Reader.java
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.google.common.collect.Lists;
import com.netflix.iceberg.CombinedScanTask;
import com.netflix.iceberg.DataFile;
import com.netflix.iceberg.FileIO;
import com.netflix.iceberg.FileScanTask;
import com.netflix.iceberg.PartitionField;
import com.netflix.iceberg.PartitionSpec;
Expand All @@ -34,7 +35,6 @@
import com.netflix.iceberg.common.DynMethods;
import com.netflix.iceberg.exceptions.RuntimeIOException;
import com.netflix.iceberg.expressions.Expression;
import com.netflix.iceberg.hadoop.HadoopInputFile;
import com.netflix.iceberg.io.CloseableIterable;
import com.netflix.iceberg.io.InputFile;
import com.netflix.iceberg.parquet.Parquet;
Expand All @@ -44,7 +44,6 @@
import com.netflix.iceberg.spark.data.SparkParquetReaders;
import com.netflix.iceberg.types.TypeUtil;
import com.netflix.iceberg.types.Types;
import org.apache.hadoop.conf.Configuration;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.catalyst.expressions.Attribute;
import org.apache.spark.sql.catalyst.expressions.AttributeReference;
Expand All @@ -67,7 +66,6 @@
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.unsafe.types.UTF8String;
import org.apache.spark.util.SerializableConfiguration;
import java.io.Closeable;
import java.io.IOException;
import java.io.Serializable;
Expand All @@ -89,7 +87,7 @@ class Reader implements DataSourceReader, SupportsPushDownFilters, SupportsPushD
private static final Filter[] NO_FILTERS = new Filter[0];

private final Table table;
private final SerializableConfiguration conf;
private final FileIO fileIo;
private StructType requestedSchema = null;
private List<Expression> filterExpressions = null;
private Filter[] pushedFilters = NO_FILTERS;
Expand All @@ -99,10 +97,10 @@ class Reader implements DataSourceReader, SupportsPushDownFilters, SupportsPushD
private StructType type = null; // cached because Spark accesses it multiple times
private List<CombinedScanTask> tasks = null; // lazy cache of tasks

Reader(Table table, Configuration conf) {
Reader(Table table, FileIO fileIo) {
this.table = table;
this.conf = new SerializableConfiguration(conf);
this.schema = table.schema();
this.fileIo = fileIo;
}

private Schema lazySchema() {
Expand Down Expand Up @@ -135,7 +133,7 @@ public List<InputPartition<InternalRow>> planInputPartitions() {

List<InputPartition<InternalRow>> readTasks = Lists.newArrayList();
for (CombinedScanTask task : tasks()) {
readTasks.add(new ReadTask(task, tableSchemaString, expectedSchemaString, conf));
readTasks.add(new ReadTask(task, tableSchemaString, expectedSchemaString, fileIo));
}

return readTasks;
Expand Down Expand Up @@ -228,22 +226,22 @@ private static class ReadTask implements InputPartition<InternalRow>, Serializab
private final CombinedScanTask task;
private final String tableSchemaString;
private final String expectedSchemaString;
private final SerializableConfiguration conf;
private final FileIO fileIo;

private transient Schema tableSchema = null;
private transient Schema expectedSchema = null;

private ReadTask(CombinedScanTask task, String tableSchemaString, String expectedSchemaString,
SerializableConfiguration conf) {
private ReadTask(
CombinedScanTask task, String tableSchemaString, String expectedSchemaString, FileIO fileIo) {
this.task = task;
this.tableSchemaString = tableSchemaString;
this.expectedSchemaString = expectedSchemaString;
this.conf = conf;
this.fileIo = fileIo;
}

@Override
public InputPartitionReader<InternalRow> createPartitionReader() {
return new TaskDataReader(task, lazyTableSchema(), lazyExpectedSchema(), conf.value());
return new TaskDataReader(task, lazyTableSchema(), lazyExpectedSchema(), fileIo);
}

private Schema lazyTableSchema() {
Expand All @@ -270,18 +268,18 @@ private static class TaskDataReader implements InputPartitionReader<InternalRow>
private final Iterator<FileScanTask> tasks;
private final Schema tableSchema;
private final Schema expectedSchema;
private final Configuration conf;
private final FileIO fileIo;

private Iterator<InternalRow> currentIterator = null;
private Closeable currentCloseable = null;
private InternalRow current = null;

public TaskDataReader(CombinedScanTask task, Schema tableSchema, Schema expectedSchema, Configuration conf) {
public TaskDataReader(CombinedScanTask task, Schema tableSchema, Schema expectedSchema, FileIO fileIo) {
this.fileIo = fileIo;
this.tasks = task.files().iterator();
this.tableSchema = tableSchema;
this.expectedSchema = expectedSchema;
this.conf = conf;
// open last because the schemas and conf must be set
// open last because the schemas and fileIo must be set
this.currentIterator = open(tasks.next());
}

Expand Down Expand Up @@ -346,17 +344,17 @@ private Iterator<InternalRow> open(FileScanTask task) {

// create joined rows and project from the joined schema to the final schema
iterSchema = TypeUtil.join(readSchema, partitionSchema);
iter = transform(open(task, readSchema, conf), joined::withLeft);
iter = transform(open(task, readSchema), joined::withLeft);

} else if (hasExtraFilterColumns) {
// add projection to the final schema
iterSchema = requiredSchema;
iter = open(task, requiredSchema, conf);
iter = open(task, requiredSchema);

} else {
// return the base iterator
iterSchema = finalSchema;
iter = open(task, finalSchema, conf);
iter = open(task, finalSchema);
}

// TODO: remove the projection by reporting the iterator's schema back to Spark
Expand Down Expand Up @@ -386,9 +384,8 @@ private static UnsafeProjection projection(Schema finalSchema, Schema readSchema
asScalaBufferConverter(attrs).asScala().toSeq());
}

private Iterator<InternalRow> open(FileScanTask task, Schema readSchema,
Configuration conf) {
InputFile location = HadoopInputFile.fromLocation(task.file().path(), conf);
private Iterator<InternalRow> open(FileScanTask task, Schema readSchema) {
InputFile location = fileIo.newInputFile(task.file().path().toString());
CloseableIterable<InternalRow> iter;
switch (task.file().format()) {
case PARQUET:
Expand Down
Loading