-
Notifications
You must be signed in to change notification settings - Fork 2.9k
Flink: use SerializableTable for source #6407
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,15 +31,14 @@ | |
| import org.apache.flink.table.api.TableSchema; | ||
| import org.apache.flink.table.data.RowData; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.SerializableTable; | ||
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.TableScan; | ||
| import org.apache.iceberg.encryption.EncryptionManager; | ||
| import org.apache.iceberg.expressions.Expression; | ||
| import org.apache.iceberg.flink.FlinkConfigOptions; | ||
| import org.apache.iceberg.flink.FlinkSchemaUtil; | ||
| import org.apache.iceberg.flink.TableLoader; | ||
| import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; | ||
| import org.apache.iceberg.io.FileIO; | ||
| import org.apache.iceberg.relocated.com.google.common.base.Preconditions; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
@@ -189,25 +188,17 @@ public FlinkInputFormat buildFormat() { | |
| Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); | ||
|
|
||
| Schema icebergSchema; | ||
| FileIO io; | ||
| EncryptionManager encryption; | ||
| if (table == null) { | ||
| // load required fields by table loader. | ||
| tableLoader.open(); | ||
| try (TableLoader loader = tableLoader) { | ||
| table = loader.loadTable(); | ||
| icebergSchema = table.schema(); | ||
| io = table.io(); | ||
| encryption = table.encryption(); | ||
| } catch (IOException e) { | ||
| throw new UncheckedIOException(e); | ||
| } | ||
| } else { | ||
| icebergSchema = table.schema(); | ||
| io = table.io(); | ||
| encryption = table.encryption(); | ||
| } | ||
|
|
||
| icebergSchema = table.schema(); | ||
| if (projectedSchema == null) { | ||
| contextBuilder.project(icebergSchema); | ||
| } else { | ||
|
|
@@ -220,7 +211,7 @@ public FlinkInputFormat buildFormat() { | |
| readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); | ||
|
|
||
| return new FlinkInputFormat( | ||
| tableLoader, icebergSchema, io, encryption, contextBuilder.build()); | ||
| (SerializableTable) SerializableTable.copyOf(table), contextBuilder.build()); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Personally I like that in this PR I see other place just use
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see your point. Let me change those input parameters explicitly.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is there a place where we specifically can not work with tables, just SerializedTables? |
||
| } | ||
|
|
||
| public DataStream<RowData> build() { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -40,6 +40,7 @@ | |
| import org.apache.flink.table.data.RowData; | ||
| import org.apache.flink.util.Preconditions; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.SerializableTable; | ||
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.expressions.Expression; | ||
| import org.apache.iceberg.flink.FlinkSchemaUtil; | ||
|
|
@@ -357,13 +358,11 @@ public IcebergSource<T> build() { | |
| if (readerFunction == null) { | ||
| RowDataReaderFunction rowDataReaderFunction = | ||
| new RowDataReaderFunction( | ||
| (SerializableTable) SerializableTable.copyOf(table), | ||
| flinkConfig, | ||
| table.schema(), | ||
| context.project(), | ||
| context.nameMapping(), | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this essentially deprecates
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I remember that when we were implementing historical queries in Hive (AS OF VERSION, AS OF TIMESTAMP) then we found that the namemapping is bound to the table, and not bound to the version. So if some specific schema evolution happens (migrate a table, rename a column, add back an old column - or something like this - sadly I do not remember the specifics) then we were not able to restore the original mapping from the current one, and we do not able to query the data. Arguably, this is a rare case, and the correct fix would be a spec change, still I thought it worth to mention.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Also, it was long ago, so we might want to double check the current situation before acting on this 😄
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. IIRC, This one (#2275) does the job. Now we can use schema Id in the snapshot to track which schema it writes. Anyway, let me revert this back since it should be in another PR even if we want to remove this. |
||
| context.caseSensitive(), | ||
| table.io(), | ||
| table.encryption()); | ||
| context.caseSensitive()); | ||
| this.readerFunction = (ReaderFunction<T>) rowDataReaderFunction; | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,15 +31,12 @@ | |
| import org.apache.iceberg.CombinedScanTask; | ||
| import org.apache.iceberg.DataFile; | ||
| import org.apache.iceberg.FileFormat; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.SerializableTable; | ||
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.TableProperties; | ||
| import org.apache.iceberg.encryption.EncryptionManager; | ||
| import org.apache.iceberg.flink.FlinkSchemaUtil; | ||
| import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; | ||
| import org.apache.iceberg.flink.sink.TaskWriterFactory; | ||
| import org.apache.iceberg.io.FileIO; | ||
| import org.apache.iceberg.io.TaskWriter; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
| import org.apache.iceberg.util.PropertyUtil; | ||
|
|
@@ -50,22 +47,14 @@ public class RowDataRewriter { | |
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(RowDataRewriter.class); | ||
|
|
||
| private final Schema schema; | ||
| private final String nameMapping; | ||
| private final FileIO io; | ||
| private final Table table; | ||
| private final boolean caseSensitive; | ||
| private final EncryptionManager encryptionManager; | ||
| private final TaskWriterFactory<RowData> taskWriterFactory; | ||
| private final String tableName; | ||
|
|
||
| public RowDataRewriter( | ||
| Table table, boolean caseSensitive, FileIO io, EncryptionManager encryptionManager) { | ||
| this.schema = table.schema(); | ||
| public RowDataRewriter(SerializableTable table, boolean caseSensitive) { | ||
| this.table = table; | ||
| this.caseSensitive = caseSensitive; | ||
| this.io = io; | ||
| this.encryptionManager = encryptionManager; | ||
| this.nameMapping = | ||
| PropertyUtil.propertyAsString(table.properties(), DEFAULT_NAME_MAPPING, null); | ||
| this.tableName = table.name(); | ||
|
|
||
| String formatString = | ||
|
|
@@ -77,20 +66,12 @@ public RowDataRewriter( | |
| RowType flinkSchema = FlinkSchemaUtil.convert(table.schema()); | ||
| this.taskWriterFactory = | ||
| new RowDataTaskWriterFactory( | ||
| SerializableTable.copyOf(table), | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Changed ctor input parameter to |
||
| flinkSchema, | ||
| Long.MAX_VALUE, | ||
| format, | ||
| table.properties(), | ||
| null, | ||
| false); | ||
| table, flinkSchema, Long.MAX_VALUE, format, table.properties(), null, false); | ||
| } | ||
|
|
||
| public List<DataFile> rewriteDataForTasks( | ||
| DataStream<CombinedScanTask> dataStream, int parallelism) throws Exception { | ||
| RewriteMap map = | ||
| new RewriteMap( | ||
| schema, nameMapping, io, caseSensitive, encryptionManager, taskWriterFactory); | ||
| RewriteMap map = new RewriteMap(table, caseSensitive, taskWriterFactory); | ||
| DataStream<List<DataFile>> ds = dataStream.map(map).setParallelism(parallelism); | ||
| return Lists.newArrayList(ds.executeAndCollect("Rewrite table :" + tableName)).stream() | ||
| .flatMap(Collection::stream) | ||
|
|
@@ -103,29 +84,20 @@ public static class RewriteMap extends RichMapFunction<CombinedScanTask, List<Da | |
| private int subTaskId; | ||
| private int attemptId; | ||
|
|
||
| private final Schema schema; | ||
| private final String nameMapping; | ||
| private final FileIO io; | ||
| private final boolean caseSensitive; | ||
| private final EncryptionManager encryptionManager; | ||
| private final Table table; | ||
| private final TaskWriterFactory<RowData> taskWriterFactory; | ||
| private final RowDataFileScanTaskReader rowDataReader; | ||
|
|
||
| public RewriteMap( | ||
| Schema schema, | ||
| String nameMapping, | ||
| FileIO io, | ||
| boolean caseSensitive, | ||
| EncryptionManager encryptionManager, | ||
| TaskWriterFactory<RowData> taskWriterFactory) { | ||
| this.schema = schema; | ||
| this.nameMapping = nameMapping; | ||
| this.io = io; | ||
| this.caseSensitive = caseSensitive; | ||
| this.encryptionManager = encryptionManager; | ||
| Table table, boolean caseSensitive, TaskWriterFactory<RowData> taskWriterFactory) { | ||
| this.table = table; | ||
| this.taskWriterFactory = taskWriterFactory; | ||
| this.rowDataReader = | ||
| new RowDataFileScanTaskReader(schema, schema, nameMapping, caseSensitive); | ||
| new RowDataFileScanTaskReader( | ||
| table.schema(), | ||
| table.schema(), | ||
| table.properties().get(DEFAULT_NAME_MAPPING), | ||
| caseSensitive); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -140,8 +112,7 @@ public void open(Configuration parameters) { | |
| public List<DataFile> map(CombinedScanTask task) throws Exception { | ||
| // Initialize the task writer. | ||
| this.writer = taskWriterFactory.create(); | ||
| try (DataIterator<RowData> iterator = | ||
| new DataIterator<>(rowDataReader, task, io, encryptionManager)) { | ||
| try (DataIterator<RowData> iterator = new DataIterator<>(table, rowDataReader, task)) { | ||
| while (iterator.hasNext()) { | ||
| RowData rowData = iterator.next(); | ||
| writer.write(rowData); | ||
|
|
||
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.
just to double check. input format is only for batch query, right? SerializableTable is read only and can't be used for long-running streaming source.
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.
Flink source uses the monitor function to monitor snapshots and forward splits to
StreamingReaderOperator,StreamingReaderOperatorusesFlinkInputFormatto open splits. Opening splits should work even the table is readonly.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.
yeah. using readonly table for reader function is fine.